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

[01/37] incubator-ignite git commit: #IGNITE-857 Implemented Mesos scheduler and framework.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-gg-10369 f23566ef0 -> a7301274d


#IGNITE-857 Implemented Mesos scheduler and framework.


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

Branch: refs/heads/ignite-gg-10369
Commit: 934bf696f6836a8db294f8fcc88717cb2698093c
Parents: 39ae0c8
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu May 7 18:25:40 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu May 7 18:26:43 2015 +0300

----------------------------------------------------------------------
 modules/mesos/README.txt                        |   4 +
 modules/mesos/licenses/apache-2.0.txt           | 202 ++++++++++++++++++
 modules/mesos/pom.xml                           |  79 +++++++
 .../apache/ignite/messo/IgniteFramework.java    |  83 ++++++++
 .../apache/ignite/messo/IgniteScheduler.java    | 213 +++++++++++++++++++
 .../org/apache/ignite/messo/package-info.java   |  22 ++
 pom.xml                                         |   1 +
 7 files changed, 604 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/934bf696/modules/mesos/README.txt
----------------------------------------------------------------------
diff --git a/modules/mesos/README.txt b/modules/mesos/README.txt
new file mode 100644
index 0000000..084ccce
--- /dev/null
+++ b/modules/mesos/README.txt
@@ -0,0 +1,4 @@
+Apache Ignite Mesos Module
+------------------------
+
+TODO
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/934bf696/modules/mesos/licenses/apache-2.0.txt
----------------------------------------------------------------------
diff --git a/modules/mesos/licenses/apache-2.0.txt b/modules/mesos/licenses/apache-2.0.txt
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/modules/mesos/licenses/apache-2.0.txt
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/934bf696/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
new file mode 100644
index 0000000..8c50085
--- /dev/null
+++ b/modules/mesos/pom.xml
@@ -0,0 +1,79 @@
+<?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-mesos</artifactId>
+    <version>1.1.0-SNAPSHOT</version>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.mesos</groupId>
+            <artifactId>mesos</artifactId>
+            <version>0.22.0</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+            <version>1.7.12</version>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <version>3.1</version>
+                <configuration>
+                    <source>1.7</source>
+                    <target>1.7</target>
+                </configuration>
+            </plugin>
+
+            <plugin>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <version>2.4.1</version>
+                <configuration>
+                    <descriptorRefs>
+                        <descriptorRef>fat-jar</descriptorRef>
+                    </descriptorRefs>
+                </configuration>
+                <executions>
+                    <execution>
+                        <id>make-assembly</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>single</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+
+        </plugins>
+    </build>
+
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/934bf696/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteFramework.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteFramework.java b/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteFramework.java
new file mode 100644
index 0000000..54d34c1
--- /dev/null
+++ b/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteFramework.java
@@ -0,0 +1,83 @@
+/*
+ * 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.messo;
+
+import com.google.protobuf.*;
+import org.apache.mesos.*;
+
+/**
+ * TODO
+ */
+public class IgniteFramework {
+    /**
+     * @param args Args
+     */
+    public static void main(String[] args) {
+        final int frameworkFailoverTimeout = 0;
+
+        Protos.FrameworkInfo.Builder frameworkBuilder = Protos.FrameworkInfo.newBuilder()
+            .setName("IgniteFramework")
+            .setUser("") // Have Mesos fill in the current user.
+            .setFailoverTimeout(frameworkFailoverTimeout); // timeout in seconds
+
+        if (System.getenv("MESOS_CHECKPOINT") != null) {
+            System.out.println("Enabling checkpoint for the framework");
+            frameworkBuilder.setCheckpoint(true);
+        }
+
+        // create the scheduler
+        final Scheduler scheduler = new IgniteScheduler();
+
+        // create the driver
+        MesosSchedulerDriver driver;
+        if (System.getenv("MESOS_AUTHENTICATE") != null) {
+            System.out.println("Enabling authentication for the framework");
+
+            if (System.getenv("DEFAULT_PRINCIPAL") == null) {
+                System.err.println("Expecting authentication principal in the environment");
+                System.exit(1);
+            }
+
+            if (System.getenv("DEFAULT_SECRET") == null) {
+                System.err.println("Expecting authentication secret in the environment");
+                System.exit(1);
+            }
+
+            Protos.Credential credential = Protos.Credential.newBuilder()
+                .setPrincipal(System.getenv("DEFAULT_PRINCIPAL"))
+                .setSecret(ByteString.copyFrom(System.getenv("DEFAULT_SECRET").getBytes()))
+                .build();
+
+            frameworkBuilder.setPrincipal(System.getenv("DEFAULT_PRINCIPAL"));
+
+            driver = new MesosSchedulerDriver(scheduler, frameworkBuilder.build(), args[0], credential);
+        }
+        else {
+            frameworkBuilder.setPrincipal("ignite-framework-java");
+
+            driver = new MesosSchedulerDriver(scheduler, frameworkBuilder.build(), args[0]);
+        }
+
+        int status = driver.run() == Protos.Status.DRIVER_STOPPED ? 0 : 1;
+
+        // Ensure that the driver process terminates.
+        driver.stop();
+
+        System.exit(status);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/934bf696/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java
new file mode 100644
index 0000000..bddcec6
--- /dev/null
+++ b/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java
@@ -0,0 +1,213 @@
+/*
+ * 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.messo;
+
+import org.apache.mesos.*;
+import org.slf4j.*;
+
+import java.util.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * TODO
+ */
+public class IgniteScheduler implements Scheduler {
+    /** Docker image name. */
+    public static final String IMAGE = "apacheignite/ignite-docker";
+
+    /** Cpus. */
+    public static final String CPUS = "cpus";
+
+    /** Mem. */
+    public static final String MEM = "mem";
+
+    /** ID generator. */
+    private AtomicInteger taskIdGenerator = new AtomicInteger();
+
+    /** Logger. */
+    private static final Logger log = LoggerFactory.getLogger(IgniteScheduler.class);
+
+    /** Min of memory required. */
+    public static final int MIN_MEMORY = 256;
+
+    /** {@inheritDoc} */
+    @Override public void registered(SchedulerDriver schedulerDriver, Protos.FrameworkID frameworkID,
+        Protos.MasterInfo masterInfo) {
+        log.info("registered() master={}:{}, framework={}", masterInfo.getIp(), masterInfo.getPort(), frameworkID);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void reregistered(SchedulerDriver schedulerDriver, Protos.MasterInfo masterInfo) {
+        log.info("reregistered");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void resourceOffers(SchedulerDriver schedulerDriver, List<Protos.Offer> offers) {
+        log.info("resourceOffers() with {} offers", offers.size());
+
+        List<Protos.OfferID> offerIDs = new ArrayList<>(offers.size());
+        List<Protos.TaskInfo> tasks = new ArrayList<>(offers.size());
+
+        for (Protos.Offer offer : offers) {
+            Pair<Double, Double> cpuMem = checkOffer(offer);
+
+            //
+            if (cpuMem == null)
+                continue;
+
+            // Generate a unique task ID.
+            Protos.TaskID taskId = Protos.TaskID.newBuilder()
+                .setValue(Integer.toString(taskIdGenerator.incrementAndGet())).build();
+
+            log.info("Launching task {}", taskId.getValue());
+
+            // Docker image info.
+            Protos.ContainerInfo.DockerInfo.Builder docker = Protos.ContainerInfo.DockerInfo.newBuilder()
+                .setImage(IMAGE)
+                .setNetwork(Protos.ContainerInfo.DockerInfo.Network.HOST);
+
+            // Container info.
+            Protos.ContainerInfo.Builder cont = Protos.ContainerInfo.newBuilder();
+            cont.setType(Protos.ContainerInfo.Type.DOCKER);
+            cont.setDocker(docker.build());
+
+            // Create task to run.
+            Protos.TaskInfo task = Protos.TaskInfo.newBuilder()
+                .setName("task " + taskId.getValue())
+                .setTaskId(taskId)
+                .setSlaveId(offer.getSlaveId())
+                .addResources(Protos.Resource.newBuilder()
+                    .setName(CPUS)
+                    .setType(Protos.Value.Type.SCALAR)
+                    .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem._1)))
+                .addResources(Protos.Resource.newBuilder()
+                    .setName(MEM)
+                    .setType(Protos.Value.Type.SCALAR)
+                    .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem._2)))
+                .setContainer(cont)
+                .setCommand(Protos.CommandInfo.newBuilder().setShell(false))
+                .build();
+
+            offerIDs.add(offer.getId());
+            tasks.add(task);
+        }
+
+        schedulerDriver.launchTasks(offerIDs, tasks, Protos.Filters.newBuilder().setRefuseSeconds(1).build());
+    }
+
+    /**
+     * Check slave resources and return resources infos.
+     *
+     * @param offer Offer request.
+     * @return Pair where first is cpus, second is memory.
+     */
+    private Pair<Double, Double> checkOffer(Protos.Offer offer) {
+        double cpus = -1;
+        double mem = -1;
+
+        for (Protos.Resource resource : offer.getResourcesList()) {
+            if (resource.getName().equals(CPUS)) {
+                if (resource.getType().equals(Protos.Value.Type.SCALAR))
+                    cpus = resource.getScalar().getValue();
+                else
+                    log.debug("Cpus resource was not a scalar: " + resource.getType().toString());
+            }
+            else if (resource.getName().equals(MEM)) {
+                if (resource.getType().equals(Protos.Value.Type.SCALAR))
+                    mem = resource.getScalar().getValue();
+                else
+                    log.debug("Mem resource was not a scalar: " + resource.getType().toString());
+            }
+            else if (resource.getName().equals("disk"))
+                log.debug("Ignoring disk resources from offer");
+        }
+
+        if (cpus < 0)
+            log.debug("No cpus resource present");
+        if (mem < 0)
+            log.debug("No mem resource present");
+
+        if (cpus >= 1 && MIN_MEMORY <= mem)
+            return new Pair<>(cpus, mem);
+        else {
+            log.info("Offer not sufficient for slave request:\n" + offer.getResourcesList().toString() +
+                "\n" + offer.getAttributesList().toString() +
+                "\nRequested for slave:\n" +
+                "  cpus:  " + cpus + "\n" +
+                "  mem:   " + mem);
+
+            return null;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void offerRescinded(SchedulerDriver schedulerDriver, Protos.OfferID offerID) {
+        log.info("offerRescinded()");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void statusUpdate(SchedulerDriver schedulerDriver, Protos.TaskStatus taskStatus) {
+        log.info("statusUpdate() task {} ", taskStatus);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void frameworkMessage(SchedulerDriver schedulerDriver, Protos.ExecutorID executorID,
+        Protos.SlaveID slaveID, byte[] bytes) {
+        log.info("frameworkMessage()");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void disconnected(SchedulerDriver schedulerDriver) {
+        log.info("disconnected()");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void slaveLost(SchedulerDriver schedulerDriver, Protos.SlaveID slaveID) {
+        log.info("slaveLost()");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void executorLost(SchedulerDriver schedulerDriver, Protos.ExecutorID executorID,
+        Protos.SlaveID slaveID, int i) {
+        log.info("executorLost()");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void error(SchedulerDriver schedulerDriver, String s) {
+        log.error("error() {}", s);
+    }
+
+    /**
+     * Tuple.
+     */
+    public static class Pair<A, B> {
+        /** */
+        public final A _1;
+
+        /** */
+        public final B _2;
+
+        /**
+         *
+         */
+        public Pair(A _1, B _2) {
+            this._1 = _1;
+            this._2 = _2;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/934bf696/modules/mesos/src/main/java/org/apache/ignite/messo/package-info.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/messo/package-info.java b/modules/mesos/src/main/java/org/apache/ignite/messo/package-info.java
new file mode 100644
index 0000000..c48ca38
--- /dev/null
+++ b/modules/mesos/src/main/java/org/apache/ignite/messo/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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 description. -->
+ * Messo Framework.
+ */
+package org.apache.ignite.messo;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/934bf696/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 9321a28..7350ae0 100644
--- a/pom.xml
+++ b/pom.xml
@@ -77,6 +77,7 @@
         <module>modules/codegen</module>
         <module>modules/gce</module>
         <module>modules/cloud</module>
+        <module>modules/mesos</module>
     </modules>
 
     <profiles>


[04/37] incubator-ignite git commit: temp commit

Posted by sb...@apache.org.
temp commit


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

Branch: refs/heads/ignite-gg-10369
Commit: 21a1514dd80e6a9083559de85da873f165390eba
Parents: fbdcb70
Author: nikolay tikhonov <nt...@gridgain.com>
Authored: Mon May 18 12:48:21 2015 +0300
Committer: nikolay tikhonov <nt...@gridgain.com>
Committed: Mon May 18 12:48:21 2015 +0300

----------------------------------------------------------------------
 .../ignite/messo/IgniteAmazonScheduler.java     |  8 ++--
 .../apache/ignite/messo/IgniteScheduler.java    | 44 +++++++++++++-------
 2 files changed, 33 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/21a1514d/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteAmazonScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteAmazonScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteAmazonScheduler.java
index 250fec6..b11e7c6 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteAmazonScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteAmazonScheduler.java
@@ -44,7 +44,7 @@ public class IgniteAmazonScheduler extends IgniteScheduler {
     }
 
     /** {@inheritDoc} */
-    @Override protected Protos.TaskInfo createTask(Protos.Offer offer, Pair<Double, Double> cpuMem,
+    @Override protected Protos.TaskInfo createTask(Protos.Offer offer, Tuple<Double, Double> cpuMem,
         Protos.TaskID taskId) {
         // Docker image info.
         Protos.ContainerInfo.DockerInfo.Builder docker = Protos.ContainerInfo.DockerInfo.newBuilder()
@@ -63,16 +63,16 @@ public class IgniteAmazonScheduler extends IgniteScheduler {
             .addResources(Protos.Resource.newBuilder()
                 .setName(CPUS)
                 .setType(Protos.Value.Type.SCALAR)
-                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem._1)))
+                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem.get1())))
             .addResources(Protos.Resource.newBuilder()
                 .setName(MEM)
                 .setType(Protos.Value.Type.SCALAR)
-                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem._2)))
+                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem.get2())))
             .setContainer(cont)
             .setCommand(Protos.CommandInfo.newBuilder()
                 .setShell(false)
                 .addArguments(STARTUP_SCRIPT)
-                .addArguments(String.valueOf(cpuMem._2.intValue()))
+                .addArguments(String.valueOf(cpuMem.get2().intValue()))
                 .addArguments(AMAZON)
                 .addArguments(accessKey)
                 .addArguments(secretKey))

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/21a1514d/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java
index a029e69..c8b577f 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java
@@ -64,7 +64,7 @@ public class IgniteScheduler implements Scheduler {
         log.info("resourceOffers() with {} offers", offers.size());
 
         for (Protos.Offer offer : offers) {
-            Pair<Double, Double> cpuMem = checkOffer(offer);
+            Tuple<Double, Double> cpuMem = checkOffer(offer);
 
             // Decline offer which doesn't match by mem or cpu.
             if (cpuMem == null) {
@@ -95,11 +95,11 @@ public class IgniteScheduler implements Scheduler {
      * @param taskId Task id.
      * @return Task.
      */
-    protected Protos.TaskInfo createTask(Protos.Offer offer, Pair<Double, Double> cpuMem, Protos.TaskID taskId) {
+    protected Protos.TaskInfo createTask(Protos.Offer offer, Tuple<Double, Double> cpuMem, Protos.TaskID taskId) {
         // Docker image info.
         Protos.ContainerInfo.DockerInfo.Builder docker = Protos.ContainerInfo.DockerInfo.newBuilder()
-                .setImage(IMAGE)
-                .setNetwork(Protos.ContainerInfo.DockerInfo.Network.HOST);
+            .setImage(IMAGE)
+            .setNetwork(Protos.ContainerInfo.DockerInfo.Network.HOST);
 
         // Container info.
         Protos.ContainerInfo.Builder cont = Protos.ContainerInfo.newBuilder();
@@ -113,16 +113,16 @@ public class IgniteScheduler implements Scheduler {
             .addResources(Protos.Resource.newBuilder()
                 .setName(CPUS)
                 .setType(Protos.Value.Type.SCALAR)
-                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem._1)))
+                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem.get2())))
             .addResources(Protos.Resource.newBuilder()
                 .setName(MEM)
                 .setType(Protos.Value.Type.SCALAR)
-                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem._2)))
+                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem.get2())))
             .setContainer(cont)
             .setCommand(Protos.CommandInfo.newBuilder()
                 .setShell(false)
                 .addArguments(STARTUP_SCRIPT)
-                .addArguments(String.valueOf(cpuMem._2.intValue())))
+                .addArguments(String.valueOf(cpuMem.get2().intValue())))
             .build();
     }
 
@@ -132,7 +132,7 @@ public class IgniteScheduler implements Scheduler {
      * @param offer Offer request.
      * @return Pair where first is cpus, second is memory.
      */
-    private Pair<Double, Double> checkOffer(Protos.Offer offer) {
+    private Tuple<Double, Double> checkOffer(Protos.Offer offer) {
         double cpus = -1;
         double mem = -1;
 
@@ -159,7 +159,7 @@ public class IgniteScheduler implements Scheduler {
             log.debug("No mem resource present");
 
         if (cpus >= 1 && MIN_MEMORY <= mem)
-            return new Pair<>(cpus, mem);
+            return new Tuple<>(cpus, mem);
         else {
             log.info("Offer not sufficient for slave request:\n" + offer.getResourcesList().toString() +
                 "\n" + offer.getAttributesList().toString() +
@@ -211,19 +211,33 @@ public class IgniteScheduler implements Scheduler {
     /**
      * Tuple.
      */
-    public static class Pair<A, B> {
+    public static class Tuple<A, B> {
         /** */
-        public final A _1;
+        private final A val1;
 
         /** */
-        public final B _2;
+        private final B val2;
 
         /**
          *
          */
-        public Pair(A _1, B _2) {
-            this._1 = _1;
-            this._2 = _2;
+        public Tuple(A val1, B val2) {
+            this.val1 = val1;
+            this.val2 = val2;
+        }
+
+        /**
+         * @return val1
+         */
+        public A get1() {
+            return val1;
+        }
+
+        /**
+         * @return val2
+         */
+        public B get2() {
+            return val2;
         }
     }
 }


[32/37] incubator-ignite git commit: #IGNITE-857 Removed log4j.

Posted by sb...@apache.org.
#IGNITE-857 Removed log4j.


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

Branch: refs/heads/ignite-gg-10369
Commit: bac7f79af721874e8d36a01471e0b0362c54e442
Parents: 73c519e
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu May 28 19:01:49 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu May 28 19:01:49 2015 +0300

----------------------------------------------------------------------
 modules/mesos/pom.xml                           |  7 --
 .../apache/ignite/mesos/ClusterProperties.java  |  7 +-
 .../apache/ignite/mesos/IgniteFramework.java    |  8 +-
 .../apache/ignite/mesos/IgniteScheduler.java    | 83 ++++++++------------
 modules/mesos/src/main/resources/log4j2.xml     | 35 ---------
 5 files changed, 40 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bac7f79a/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index 3cf9028..eca4fa9 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -30,7 +30,6 @@
     <properties>
         <jetty.version>9.2.10.v20150310</jetty.version>
         <mesos.version>0.22.0</mesos.version>
-        <log4j.version>2.0.2</log4j.version>
     </properties>
 
     <dependencies>
@@ -41,12 +40,6 @@
         </dependency>
 
         <dependency>
-            <groupId>org.apache.logging.log4j</groupId>
-            <artifactId>log4j-core</artifactId>
-            <version>${log4j.version}</version>
-        </dependency>
-
-        <dependency>
             <groupId>org.eclipse.jetty</groupId>
             <artifactId>jetty-server</artifactId>
             <version>${jetty.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bac7f79a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
index 956dd6e..6663625 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
@@ -17,11 +17,10 @@
 
 package org.apache.ignite.mesos;
 
-import org.apache.logging.log4j.*;
-
 import java.io.*;
 import java.net.*;
 import java.util.*;
+import java.util.logging.*;
 import java.util.regex.*;
 
 /**
@@ -29,7 +28,7 @@ import java.util.regex.*;
  */
 public class ClusterProperties {
     /** */
-    private static final Logger log = LogManager.getLogger(ClusterProperties.class);
+    private static final Logger log = Logger.getLogger(ClusterProperties.class.getSimpleName());
 
     /** Unlimited. */
     public static final double UNLIMITED = Double.MAX_VALUE;
@@ -448,7 +447,7 @@ public class ClusterProperties {
                     prop.hostnameConstraint = Pattern.compile(pattern);
                 }
                 catch (PatternSyntaxException e) {
-                    log.warn("IGNITE_HOSTNAME_CONSTRAINT has invalid pattern. It will be ignore.", e);
+                    log.log(Level.WARNING, "IGNITE_HOSTNAME_CONSTRAINT has invalid pattern. It will be ignore.", e);
                 }
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bac7f79a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
index c30d206..3d582d9 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
@@ -19,17 +19,17 @@ package org.apache.ignite.mesos;
 
 import com.google.protobuf.*;
 import org.apache.ignite.mesos.resource.*;
-import org.apache.logging.log4j.*;
 import org.apache.mesos.*;
 
 import java.net.*;
+import java.util.logging.*;
 
 /**
  * Ignite mesos framework.
  */
 public class IgniteFramework {
     /** */
-    public static final Logger log = LogManager.getLogger(IgniteFramework.class);
+    public static final Logger log = Logger.getLogger(IgniteFramework.class.getSimpleName());
 
     /** Framework name. */
     public static final String IGNITE_FRAMEWORK_NAME = "Ignite";
@@ -80,13 +80,13 @@ public class IgniteFramework {
             log.info("Enabling authentication for the framework");
 
             if (System.getenv("DEFAULT_PRINCIPAL") == null) {
-                log.error("Expecting authentication principal in the environment");
+                log.log(Level.SEVERE, "Expecting authentication principal in the environment");
 
                 System.exit(1);
             }
 
             if (System.getenv("DEFAULT_SECRET") == null) {
-                log.error("Expecting authentication secret in the environment");
+                log.log(Level.SEVERE, "Expecting authentication secret in the environment");
 
                 System.exit(1);
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bac7f79a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index 6bd3aa7..fbb9994 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -18,11 +18,11 @@
 package org.apache.ignite.mesos;
 
 import org.apache.ignite.mesos.resource.*;
-import org.apache.logging.log4j.*;
 import org.apache.mesos.*;
 
 import java.util.*;
 import java.util.concurrent.atomic.*;
+import java.util.logging.*;
 
 /**
  * Ignite scheduler receives offers from Mesos and decides how many resources will be occupied.
@@ -44,10 +44,7 @@ public class IgniteScheduler implements Scheduler {
     public static final String DELIM = ",";
 
     /** Logger. */
-    private static final Logger log = LogManager.getLogger(IgniteScheduler.class);
-
-    /** Mutex. */
-    private final Object mux = new Object();
+    private static final Logger log = Logger.getLogger(IgniteScheduler.class.getSimpleName());
 
     /** ID generator. */
     private AtomicInteger taskIdGenerator = new AtomicInteger();
@@ -71,33 +68,27 @@ public class IgniteScheduler implements Scheduler {
     }
 
     /** {@inheritDoc} */
-    @Override public void resourceOffers(SchedulerDriver schedulerDriver, List<Protos.Offer> offers) {
-        log.debug("Offers resources: {} ", offers.size());
+    @Override public synchronized void resourceOffers(SchedulerDriver schedulerDriver, List<Protos.Offer> offers) {
+        log.log(Level.FINE, "Offers resources: {0}", offers.size());
 
         for (Protos.Offer offer : offers) {
-            Protos.TaskID taskId;
-            Protos.TaskInfo task;
-            IgniteTask igniteTask;
+            IgniteTask igniteTask = checkOffer(offer);
 
-            synchronized (mux) {
-                igniteTask = checkOffer(offer);
+            // Decline offer which doesn't match by mem or cpu.
+            if (igniteTask == null) {
+                schedulerDriver.declineOffer(offer.getId());
 
-                // Decline offer which doesn't match by mem or cpu.
-                if (igniteTask == null) {
-                    schedulerDriver.declineOffer(offer.getId());
+                continue;
+            }
 
-                    continue;
-                }
+            // Generate a unique task ID.
+            Protos.TaskID taskId = Protos.TaskID.newBuilder()
+                .setValue(Integer.toString(taskIdGenerator.incrementAndGet())).build();
 
-                // Generate a unique task ID.
-                taskId = Protos.TaskID.newBuilder()
-                    .setValue(Integer.toString(taskIdGenerator.incrementAndGet())).build();
+            log.log(Level.INFO, "Launching task: {0}", igniteTask);
 
-                log.info("Launching task: {}", igniteTask);
-
-                // Create task to run.
-                task = createTask(offer, igniteTask, taskId);
-            }
+            // Create task to run.
+            Protos.TaskInfo task = createTask(offer, igniteTask, taskId);
 
             try {
                 schedulerDriver.launchTasks(Collections.singletonList(offer.getId()),
@@ -105,14 +96,13 @@ public class IgniteScheduler implements Scheduler {
                     Protos.Filters.newBuilder().setRefuseSeconds(1).build());
             }
             catch (RuntimeException e) {
-                log.error("Failed launch task. Task id: {}. Task info: {}", taskId, task);
+                log.log(Level.SEVERE, "Failed launch task. Task id: {0}. Task info: {1}",
+                    new Object[]{taskId, task, e});
 
                 throw e;
             }
 
-            synchronized (mux) {
-                tasks.put(taskId.getValue(), igniteTask);
-            }
+            tasks.put(taskId.getValue(), igniteTask);
         }
     }
 
@@ -124,7 +114,7 @@ public class IgniteScheduler implements Scheduler {
      * @param taskId Task id.
      * @return Task.
      */
-    protected Protos.TaskInfo createTask(Protos.Offer offer, IgniteTask igniteTask, Protos.TaskID taskId) {
+    private Protos.TaskInfo createTask(Protos.Offer offer, IgniteTask igniteTask, Protos.TaskID taskId) {
         String cfgUrl = clusterProps.igniteConfigUrl() != null ?
             clusterProps.igniteConfigUrl() : resourceProvider.igniteConfigUrl();
 
@@ -189,8 +179,6 @@ public class IgniteScheduler implements Scheduler {
      * @return Address running nodes.
      */
     private String getAddress(String address) {
-        assert Thread.holdsLock(mux);
-
         if (tasks.isEmpty()) {
             if (address != null && !address.isEmpty())
                 return address + DEFAULT_PORT;
@@ -213,8 +201,6 @@ public class IgniteScheduler implements Scheduler {
      * @return Ignite task description.
      */
     private IgniteTask checkOffer(Protos.Offer offer) {
-        assert Thread.holdsLock(mux);
-
         // Check limit on running nodes.
         if (clusterProps.instances() <= tasks.size())
             return null;
@@ -234,24 +220,24 @@ public class IgniteScheduler implements Scheduler {
                 if (resource.getType().equals(Protos.Value.Type.SCALAR))
                     cpus = resource.getScalar().getValue();
                 else
-                    log.debug("Cpus resource was not a scalar: {}" + resource.getType());
+                    log.log(Level.FINE, "Cpus resource was not a scalar: {0}" + resource.getType());
             }
             else if (resource.getName().equals(MEM)) {
                 if (resource.getType().equals(Protos.Value.Type.SCALAR))
                     mem = resource.getScalar().getValue();
                 else
-                    log.debug("Mem resource was not a scalar: {}", resource.getType());
+                    log.log(Level.FINE, "Mem resource was not a scalar: {0}", resource.getType());
             }
             else if (resource.getName().equals(DISK))
                 if (resource.getType().equals(Protos.Value.Type.SCALAR))
                     disk = resource.getScalar().getValue();
                 else
-                    log.debug("Disk resource was not a scalar: {}", resource.getType());
+                    log.log(Level.FINE, "Disk resource was not a scalar: {0}", resource.getType());
         }
 
         // Check that slave satisfies min requirements.
         if (cpus < clusterProps.minCpuPerNode() || mem < clusterProps.minMemoryPerNode()) {
-            log.debug("Offer not sufficient for slave request: {}", offer.getResourcesList());
+            log.log(Level.FINE, "Offer not sufficient for slave request: {0}", offer.getResourcesList());
 
             return null;
         }
@@ -273,7 +259,7 @@ public class IgniteScheduler implements Scheduler {
 
         if ((clusterProps.cpusPerNode() != ClusterProperties.UNLIMITED && clusterProps.cpusPerNode() != cpus)
             || (clusterProps.memoryPerNode() != ClusterProperties.UNLIMITED && clusterProps.memoryPerNode() != mem)) {
-            log.debug("Offer not sufficient for slave request: {}", offer.getResourcesList());
+            log.log(Level.FINE, "Offer not sufficient for slave request: {0}", offer.getResourcesList());
 
             return null;
         }
@@ -281,28 +267,25 @@ public class IgniteScheduler implements Scheduler {
         if (cpus > 0 && mem > 0)
             return new IgniteTask(offer.getHostname(), cpus, mem, disk);
         else {
-            log.debug("Offer not sufficient for slave request: {}", offer.getResourcesList());
+            log.log(Level.FINE, "Offer not sufficient for slave request: {0}", offer.getResourcesList());
 
             return null;
         }
     }
 
     /** {@inheritDoc} */
-    @Override public void statusUpdate(SchedulerDriver schedulerDriver, Protos.TaskStatus taskStatus) {
+    @Override public synchronized void statusUpdate(SchedulerDriver schedulerDriver, Protos.TaskStatus taskStatus) {
         final String taskId = taskStatus.getTaskId().getValue();
 
-        log.info("Received update event task: {} is in state: {}", taskId, taskStatus.getState());
+        log.log(Level.INFO, "Received update event task: {0} is in state: {1}",
+            new Object[]{taskId, taskStatus.getState()});
 
         if (taskStatus.getState().equals(Protos.TaskState.TASK_FAILED)
             || taskStatus.getState().equals(Protos.TaskState.TASK_ERROR)
             || taskStatus.getState().equals(Protos.TaskState.TASK_FINISHED)
             || taskStatus.getState().equals(Protos.TaskState.TASK_KILLED)
             || taskStatus.getState().equals(Protos.TaskState.TASK_LOST)) {
-            IgniteTask failedTask;
-
-            synchronized (mux) {
-                failedTask = tasks.remove(taskId);
-            }
+            IgniteTask failedTask = tasks.remove(taskId);
 
             if (failedTask != null) {
                 List<Protos.Request> requests = new ArrayList<>();
@@ -335,8 +318,8 @@ public class IgniteScheduler implements Scheduler {
     /** {@inheritDoc} */
     @Override public void registered(SchedulerDriver schedulerDriver, Protos.FrameworkID frameworkID,
         Protos.MasterInfo masterInfo) {
-        log.info("Scheduler registered. Master: {}:{}, framework={}", masterInfo.getIp(), masterInfo.getPort(),
-            frameworkID);
+        log.log(Level.INFO, "Scheduler registered. Master: {0}:{1}, framework={2}", new Object[]{masterInfo.getIp(),
+            masterInfo.getPort(), frameworkID});
     }
 
     /** {@inheritDoc} */
@@ -346,7 +329,7 @@ public class IgniteScheduler implements Scheduler {
 
     /** {@inheritDoc} */
     @Override public void error(SchedulerDriver schedulerDriver, String s) {
-        log.error("Failed. Error message: {}", s);
+        log.log(Level.SEVERE, "Failed. Error message: {0}", s);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bac7f79a/modules/mesos/src/main/resources/log4j2.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/resources/log4j2.xml b/modules/mesos/src/main/resources/log4j2.xml
deleted file mode 100644
index d66a83f..0000000
--- a/modules/mesos/src/main/resources/log4j2.xml
+++ /dev/null
@@ -1,35 +0,0 @@
-<?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.
--->
-
-<configuration strict="true" status="OFF" monitorInterval="30"  shutdownHook="disable">
-
-    <appenders>
-        <appender type="Console" name="STDOUT">
-            <layout type="PatternLayout" pattern="%d %p %C{1.} [%t] %m%n %ex"/>
-        </appender>
-    </appenders>
-
-    <loggers>
-        <root level="info">
-            <appender-ref ref="STDOUT"/>
-        </root>
-
-    </loggers>
-
-</configuration>
\ No newline at end of file


[30/37] incubator-ignite git commit: #IGNITE-857 Fixed review notes.

Posted by sb...@apache.org.
#IGNITE-857 Fixed review notes.


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

Branch: refs/heads/ignite-gg-10369
Commit: 73c519e4fae403c4592cfdf75158ad1df5cdf8c0
Parents: 3a0b5da
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu May 28 18:12:59 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu May 28 18:12:59 2015 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/mesos/IgniteScheduler.java   | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/73c519e4/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index 263123c..6bd3aa7 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -104,8 +104,10 @@ public class IgniteScheduler implements Scheduler {
                     Collections.singletonList(task),
                     Protos.Filters.newBuilder().setRefuseSeconds(1).build());
             }
-            catch (Exception e) {
+            catch (RuntimeException e) {
                 log.error("Failed launch task. Task id: {}. Task info: {}", taskId, task);
+
+                throw e;
             }
 
             synchronized (mux) {


[20/37] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-857' into ignite-857

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-857' into ignite-857


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

Branch: refs/heads/ignite-gg-10369
Commit: 8574a19bc010e73cccecfa1cd648709dc55b47aa
Parents: fd6f398 8deb577
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed May 27 19:16:25 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed May 27 19:16:25 2015 +0300

----------------------------------------------------------------------
 modules/mesos/licenses/jetty-epl-license.txt    |  69 +++
 modules/mesos/pom.xml                           |  42 +-
 .../apache/ignite/mesos/ClusterProperties.java  | 502 +++++++++++++++++++
 .../apache/ignite/mesos/IgniteFramework.java    | 119 +++++
 .../apache/ignite/mesos/IgniteScheduler.java    | 349 +++++++++++++
 .../org/apache/ignite/mesos/IgniteTask.java     |  86 ++++
 .../org/apache/ignite/mesos/package-info.java   |  22 +
 .../ignite/mesos/resource/IgniteProvider.java   | 234 +++++++++
 .../ignite/mesos/resource/JettyServer.java      |  61 +++
 .../ignite/mesos/resource/ResourceHandler.java  | 142 ++++++
 .../ignite/mesos/resource/ResourceProvider.java | 120 +++++
 .../ignite/mesos/resource/package-info.java     |  22 +
 .../ignite/messo/IgniteAmazonScheduler.java     |  81 ---
 .../apache/ignite/messo/IgniteFramework.java    | 108 ----
 .../apache/ignite/messo/IgniteScheduler.java    | 229 ---------
 .../org/apache/ignite/messo/package-info.java   |  22 -
 .../main/resources/ignite-default-config.xml    |  35 ++
 modules/mesos/src/main/resources/log4j2.xml     |  35 ++
 .../org/apache/ignite/IgniteMesosTestSuite.java |  38 ++
 .../ignite/mesos/IgniteSchedulerSelfTest.java   | 420 ++++++++++++++++
 20 files changed, 2293 insertions(+), 443 deletions(-)
----------------------------------------------------------------------



[22/37] incubator-ignite git commit: # ignite-857 minor

Posted by sb...@apache.org.
# ignite-857 minor


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

Branch: refs/heads/ignite-gg-10369
Commit: a346a3c20f6b15bcff140e6a1ff50ad178275eef
Parents: b7f554a
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu May 28 09:36:29 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu May 28 09:36:29 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/mesos/resource/JettyServer.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a346a3c2/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/JettyServer.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/JettyServer.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/JettyServer.java
index fb27963..446ac77 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/JettyServer.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/JettyServer.java
@@ -44,7 +44,7 @@ public class JettyServer {
             server.start();
         }
         else
-            throw new IllegalStateException("Failed. Jetty server has been started already.");
+            throw new IllegalStateException("Jetty server has already been started.");
     }
 
     /**
@@ -56,6 +56,6 @@ public class JettyServer {
         if (server != null)
             server.stop();
         else
-            throw new IllegalStateException("Failed. Jetty server has not been started yet.");
+            throw new IllegalStateException("Jetty server has not yet been started.");
     }
 }


[16/37] incubator-ignite git commit: #IGNITE-857 Code format

Posted by sb...@apache.org.
#IGNITE-857 Code format


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

Branch: refs/heads/ignite-gg-10369
Commit: b7ef1a83d9bf6cec33d2c8a36ba163f281a55a41
Parents: 29710de
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed May 27 12:26:08 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed May 27 12:26:08 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/mesos/ClusterProperties.java  | 28 ++++++++++++++++++--
 .../apache/ignite/mesos/IgniteScheduler.java    | 14 +++++-----
 .../ignite/mesos/resource/IgniteProvider.java   |  4 +--
 .../ignite/mesos/resource/ResourceHandler.java  |  4 +--
 4 files changed, 37 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b7ef1a83/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
index e659509..c23fcea 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
@@ -38,6 +38,15 @@ public class ClusterProperties {
     private String mesosUrl = DEFAULT_MESOS_MASTER_URL;
 
     /** */
+    public static final String IGNITE_CLUSTER_NAME = "IGNITE_CLUSTER_NAME";
+
+    /** */
+    public static final String DEFAULT_CLUSTER_NAME = "ignite-cluster";
+
+    /** Mesos master url. */
+    private String clusterName = DEFAULT_CLUSTER_NAME;
+
+    /** */
     public static final String IGNITE_HTTP_SERVER_HOST = "IGNITE_HTTP_SERVER_HOST";
 
     /** Http server host. */
@@ -166,6 +175,13 @@ public class ClusterProperties {
     }
 
     /**
+     * @return Cluster name.
+     */
+    public String clusterName() {
+        return clusterName;
+    }
+
+    /**
      * @return CPU count limit.
      */
     public double cpus(){
@@ -350,8 +366,16 @@ public class ClusterProperties {
             prop.mesosUrl = getStringProperty(MESOS_MASTER_URL, props, DEFAULT_MESOS_MASTER_URL);
 
             prop.httpServerHost = getStringProperty(IGNITE_HTTP_SERVER_HOST, props, getNonLoopbackAddress());
-            prop.httpServerPort = Integer.valueOf(getStringProperty(IGNITE_HTTP_SERVER_PORT, props,
-                DEFAULT_HTTP_SERVER_PORT));
+
+            String port = System.getProperty("PORT0");
+
+            if (port != null && !port.isEmpty())
+                prop.httpServerPort = Integer.valueOf(port);
+            else
+                prop.httpServerPort = Integer.valueOf(getStringProperty(IGNITE_HTTP_SERVER_PORT, props,
+                    DEFAULT_HTTP_SERVER_PORT));
+
+            prop.clusterName = getStringProperty(IGNITE_CLUSTER_NAME, props, DEFAULT_CLUSTER_NAME);
 
             prop.userLibsUrl = getStringProperty(IGNITE_USERS_LIBS_URL, props, null);
             prop.ignitePackageUrl = getStringProperty(IGNITE_PACKAGE_URL, props, null);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b7ef1a83/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index ad48189..05d8c4f 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -87,7 +87,7 @@ public class IgniteScheduler implements Scheduler {
 
                 // Generate a unique task ID.
                 Protos.TaskID taskId = Protos.TaskID.newBuilder()
-                    .setValue(Integer.toString(taskIdGenerator.incrementAndGet())).build();
+                    .setValue(taskIdGenerator.incrementAndGet() + "-" + clusterProps.clusterName()).build();
 
                 log.info("Launching task: [{}]", igniteTask);
 
@@ -143,10 +143,10 @@ public class IgniteScheduler implements Scheduler {
                     builder.addUris(Protos.CommandInfo.URI.newBuilder().setValue(url));
 
                 builder.setValue("cp *.jar ./gridgain-community-*/libs/ "
-                        + "&& ./gridgain-community-*/bin/ignite.sh "
-                        + resourceProvider.configName()
-                        + " -J-Xmx" + String.valueOf((int)igniteTask.mem() + "m")
-                        + " -J-Xms" + String.valueOf((int)igniteTask.mem()) + "m");
+                    + "&& ./gridgain-community-*/bin/ignite.sh "
+                    + resourceProvider.configName()
+                    + " -J-Xmx" + String.valueOf((int)igniteTask.mem() + "m")
+                    + " -J-Xms" + String.valueOf((int)igniteTask.mem()) + "m");
             }
         }
         else
@@ -172,7 +172,7 @@ public class IgniteScheduler implements Scheduler {
                 .setName(DISK)
                 .setType(Protos.Value.Type.SCALAR)
                 .setScalar(Protos.Value.Scalar.newBuilder().setValue(igniteTask.disk())))
-                .build();
+            .build();
     }
 
     /**
@@ -231,7 +231,7 @@ public class IgniteScheduler implements Scheduler {
         }
 
         // Check that slave satisfies min requirements.
-        if (cpus < clusterProps.minCpuPerNode() || mem < clusterProps.minMemoryPerNode() ) {
+        if (cpus < clusterProps.minCpuPerNode() || mem < clusterProps.minMemoryPerNode()) {
             log.debug("Offer not sufficient for slave request: {}", offer.getResourcesList());
 
             return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b7ef1a83/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/IgniteProvider.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/IgniteProvider.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/IgniteProvider.java
index 2887112..f459e5d 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/IgniteProvider.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/IgniteProvider.java
@@ -155,7 +155,8 @@ public class IgniteProvider {
                 outFile.close();
 
                 return parseVersion(redirectUrl);
-            } else if (code == 304)
+            }
+            else if (code == 304)
                 // This version is latest.
                 return currentVersion;
             else
@@ -199,7 +200,6 @@ public class IgniteProvider {
         }
     }
 
-
     /**
      * @return Download folder.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b7ef1a83/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceHandler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceHandler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceHandler.java
index ea883e3..cb8c773 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceHandler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceHandler.java
@@ -73,7 +73,7 @@ public class ResourceHandler extends AbstractHandler {
 
         String[] path = url.split("/");
 
-        String fileName = path[path.length -1];
+        String fileName = path[path.length - 1];
 
         String servicePath = url.substring(0, url.length() - fileName.length());
 
@@ -131,7 +131,7 @@ public class ResourceHandler extends AbstractHandler {
      * @throws IOException If failed.
      */
     private static void handleRequest(HttpServletResponse response, String type, InputStream stream,
-                                      String attachmentName) throws IOException {
+        String attachmentName) throws IOException {
         response.setContentType(type);
         response.setHeader("Content-Disposition", "attachment; filename=\"" + attachmentName + "\"");
 


[23/37] incubator-ignite git commit: #IGNITE-857 Added comma separate limit.

Posted by sb...@apache.org.
#IGNITE-857 Added comma separate limit.


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

Branch: refs/heads/ignite-gg-10369
Commit: 07a10952093f8f4c7ce432413bb582c6ab96dc26
Parents: 8deb577
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu May 28 11:15:55 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu May 28 11:15:55 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/mesos/ClusterProperties.java  | 18 +++++++
 .../apache/ignite/mesos/IgniteScheduler.java    | 34 +++++++-----
 .../ignite/mesos/IgniteSchedulerSelfTest.java   | 56 ++++++++++++++++++--
 3 files changed, 91 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/07a10952/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
index 9f0b304..944735e 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
@@ -215,6 +215,13 @@ public class ClusterProperties {
     }
 
     /**
+     * Set CPU count limit.
+     */
+    public void cpusPerNode(double cpu) {
+        this.cpuPerNode = cpu;
+    }
+
+    /**
      * @return mem limit.
      */
     public double memory() {
@@ -223,6 +230,8 @@ public class ClusterProperties {
 
     /**
      * Set mem limit.
+     *
+     * @param mem Memory.
      */
     public void memory(double mem) {
         this.mem = mem;
@@ -236,6 +245,15 @@ public class ClusterProperties {
     }
 
     /**
+     * Set mem limit.
+     *
+     * @param mem Memory.
+     */
+    public void memoryPerNode(double mem) {
+         this.memPerNode = mem;
+    }
+
+    /**
      * @return disk limit.
      */
     public double disk() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/07a10952/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index 17daf45..e833025 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -29,7 +29,7 @@ import java.util.concurrent.atomic.*;
  */
 public class IgniteScheduler implements Scheduler {
     /** Cpus. */
-    public static final String CPUS = "cpus";
+    public static final String CPU = "cpus";
 
     /** Mem. */
     public static final String MEM = "mem";
@@ -126,14 +126,17 @@ public class IgniteScheduler implements Scheduler {
             .addUris(Protos.CommandInfo.URI.newBuilder()
                 .setValue(cfgUrl));
 
-        if (clusterProps.usersLibsUrl() != null)
-            builder.addUris(Protos.CommandInfo.URI.newBuilder()
-                .setValue(clusterProps.usersLibsUrl())
-                .setExtract(true));
-        else if (resourceProvider.resourceUrl() != null) {
-            for (String url : resourceProvider.resourceUrl())
-                builder.addUris(Protos.CommandInfo.URI.newBuilder().setValue(url));
-        }
+        // Collection user's libs.
+        Collection<String> usersLibs = new ArrayList<>();
+
+        if (clusterProps.usersLibsUrl() != null && !clusterProps.usersLibsUrl().isEmpty())
+            Collections.addAll(usersLibs, clusterProps.usersLibsUrl().split(DELIM));
+
+        if (resourceProvider.resourceUrl() != null && !resourceProvider.resourceUrl().isEmpty())
+            usersLibs.addAll(resourceProvider.resourceUrl());
+
+        for (String url : usersLibs)
+            builder.addUris(Protos.CommandInfo.URI.newBuilder().setValue(url));
 
         String cfgName = resourceProvider.configName();
 
@@ -155,7 +158,7 @@ public class IgniteScheduler implements Scheduler {
             .setSlaveId(offer.getSlaveId())
             .setCommand(builder)
             .addResources(Protos.Resource.newBuilder()
-                .setName(CPUS)
+                .setName(CPU)
                 .setType(Protos.Value.Type.SCALAR)
                 .setScalar(Protos.Value.Scalar.newBuilder().setValue(igniteTask.cpuCores())))
             .addResources(Protos.Resource.newBuilder()
@@ -210,7 +213,7 @@ public class IgniteScheduler implements Scheduler {
 
         // Collect resource on slave.
         for (Protos.Resource resource : offer.getResourcesList()) {
-            if (resource.getName().equals(CPUS)) {
+            if (resource.getName().equals(CPU)) {
                 if (resource.getType().equals(Protos.Value.Type.SCALAR))
                     cpus = resource.getScalar().getValue();
                 else
@@ -251,6 +254,13 @@ public class IgniteScheduler implements Scheduler {
         mem = Math.min(clusterProps.memory() - totalMem, Math.min(mem, clusterProps.memoryPerNode()));
         disk = Math.min(clusterProps.disk() - totalDisk, Math.min(disk, clusterProps.diskPerNode()));
 
+        if ((clusterProps.cpusPerNode() != ClusterProperties.UNLIMITED && clusterProps.cpusPerNode() != cpus)
+            || (clusterProps.memoryPerNode() != ClusterProperties.UNLIMITED && clusterProps.memoryPerNode() != mem)) {
+            log.debug("Offer not sufficient for slave request: {}", offer.getResourcesList());
+
+            return null;
+        }
+
         if (cpus > 0 && mem > 0)
             return new IgniteTask(offer.getHostname(), cpus, mem, disk);
         else {
@@ -284,7 +294,7 @@ public class IgniteScheduler implements Scheduler {
                             .setScalar(Protos.Value.Scalar.newBuilder().setValue(failedTask.mem())))
                         .addResources(Protos.Resource.newBuilder()
                             .setType(Protos.Value.Type.SCALAR)
-                            .setName(CPUS)
+                            .setName(CPU)
                             .setScalar(Protos.Value.Scalar.newBuilder().setValue(failedTask.cpuCores())))
                         .build();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/07a10952/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java b/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
index 337b47c..13855b5 100644
--- a/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
+++ b/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
@@ -71,7 +71,7 @@ public class IgniteSchedulerSelfTest extends TestCase {
 
         Protos.TaskInfo taskInfo = mock.launchedTask.iterator().next();
 
-        assertEquals(4.0, resources(taskInfo.getResourcesList(), IgniteScheduler.CPUS));
+        assertEquals(4.0, resources(taskInfo.getResourcesList(), IgniteScheduler.CPU));
         assertEquals(1024.0, resources(taskInfo.getResourcesList(), IgniteScheduler.MEM));
     }
 
@@ -95,7 +95,7 @@ public class IgniteSchedulerSelfTest extends TestCase {
 
         Protos.TaskInfo taskInfo = mock.launchedTask.iterator().next();
 
-        assertEquals(2.0, resources(taskInfo.getResourcesList(), IgniteScheduler.CPUS));
+        assertEquals(2.0, resources(taskInfo.getResourcesList(), IgniteScheduler.CPU));
         assertEquals(1024.0, resources(taskInfo.getResourcesList(), IgniteScheduler.MEM));
 
         mock.clear();
@@ -130,7 +130,7 @@ public class IgniteSchedulerSelfTest extends TestCase {
 
         Protos.TaskInfo taskInfo = mock.launchedTask.iterator().next();
 
-        assertEquals(4.0, resources(taskInfo.getResourcesList(), IgniteScheduler.CPUS));
+        assertEquals(4.0, resources(taskInfo.getResourcesList(), IgniteScheduler.CPU));
         assertEquals(512.0, resources(taskInfo.getResourcesList(), IgniteScheduler.MEM));
 
         mock.clear();
@@ -168,7 +168,7 @@ public class IgniteSchedulerSelfTest extends TestCase {
 
             Protos.TaskInfo taskInfo = mock.launchedTask.iterator().next();
 
-            totalCpu += resources(taskInfo.getResourcesList(), IgniteScheduler.CPUS);
+            totalCpu += resources(taskInfo.getResourcesList(), IgniteScheduler.CPU);
             totalMem += resources(taskInfo.getResourcesList(), IgniteScheduler.MEM);
 
             mock.clear();
@@ -254,6 +254,52 @@ public class IgniteSchedulerSelfTest extends TestCase {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testPerNode() throws Exception {
+        Protos.Offer offer = createOffer("hostname", 8, 1024);
+
+        DriverMock mock = new DriverMock();
+
+        ClusterProperties clustProp = new ClusterProperties();
+        clustProp.memoryPerNode(1024);
+        clustProp.cpusPerNode(2);
+
+        scheduler.setClusterProps(clustProp);
+
+        scheduler.resourceOffers(mock, Collections.singletonList(offer));
+
+        assertNotNull(mock.launchedTask);
+
+        Protos.TaskInfo taskInfo = mock.launchedTask.iterator().next();
+
+        assertEquals(2.0, resources(taskInfo.getResourcesList(), IgniteScheduler.CPU));
+        assertEquals(1024.0, resources(taskInfo.getResourcesList(), IgniteScheduler.MEM));
+
+        mock.clear();
+
+        offer = createOffer("hostname", 1, 2048);
+
+        scheduler.resourceOffers(mock, Collections.singletonList(offer));
+
+        assertNull(mock.launchedTask);
+
+        assertNotNull(mock.declinedOffer);
+        assertEquals(offer.getId(), mock.declinedOffer);
+
+        mock.clear();
+
+        offer = createOffer("hostname", 4, 512);
+
+        scheduler.resourceOffers(mock, Collections.singletonList(offer));
+
+        assertNull(mock.launchedTask);
+
+        assertNotNull(mock.declinedOffer);
+        assertEquals(offer.getId(), mock.declinedOffer);
+    }
+
+    /**
      * @param resourceType Resource type.
      * @return Value.
      */
@@ -280,7 +326,7 @@ public class IgniteSchedulerSelfTest extends TestCase {
             .setHostname(hostname)
             .addResources(Protos.Resource.newBuilder()
                 .setType(Protos.Value.Type.SCALAR)
-                .setName(IgniteScheduler.CPUS)
+                .setName(IgniteScheduler.CPU)
                 .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpu).build())
                 .build())
             .addResources(Protos.Resource.newBuilder()


[05/37] incubator-ignite git commit: Merge branches 'ignite-857' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-857

Posted by sb...@apache.org.
Merge branches 'ignite-857' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-857


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

Branch: refs/heads/ignite-gg-10369
Commit: fd6f3989fdca948f386232eb2f540cb79b179153
Parents: fbdcb70 15d55b1
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Mon May 18 17:05:15 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Mon May 18 17:05:15 2015 +0300

----------------------------------------------------------------------
 assembly/release-base.xml                       |    4 +-
 bin/ignite-schema-import.bat                    |    2 +-
 bin/ignite-schema-import.sh                     |    2 +-
 bin/ignite.bat                                  |    2 +-
 bin/ignite.sh                                   |    2 +-
 bin/ignitevisorcmd.bat                          |    2 +-
 bin/ignitevisorcmd.sh                           |    2 +-
 bin/include/build-classpath.bat                 |   46 +
 bin/include/build-classpath.sh                  |   71 +
 bin/include/functions.sh                        |    2 +-
 bin/include/target-classpath.bat                |   46 -
 bin/include/target-classpath.sh                 |   71 -
 examples/pom.xml                                |    2 +-
 .../streaming/StreamTransformerExample.java     |    4 +-
 .../streaming/StreamVisitorExample.java         |    4 +-
 .../ignite/examples/streaming/package-info.java |    1 -
 .../streaming/wordcount/CacheConfig.java        |    2 +-
 .../streaming/wordcount/QueryWords.java         |   12 +-
 .../streaming/wordcount/StreamWords.java        |   12 +-
 .../streaming/wordcount/package-info.java       |    1 -
 .../socket/WordsSocketStreamerClient.java       |   82 +
 .../socket/WordsSocketStreamerServer.java       |  124 +
 .../wordcount/socket/package-info.java          |   21 +
 modules/aop/pom.xml                             |    2 +-
 modules/aws/pom.xml                             |    2 +-
 modules/clients/pom.xml                         |    2 +-
 modules/cloud/pom.xml                           |    4 +-
 .../TcpDiscoveryCloudIpFinderSelfTest.java      |    2 -
 modules/codegen/pom.xml                         |    2 +-
 .../ignite/codegen/MessageCodeGenerator.java    |    4 +-
 modules/core/pom.xml                            |    2 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |  136 +-
 .../internal/direct/DirectByteBufferStream.java |    4 +-
 .../internal/interop/InteropBootstrap.java      |   34 +
 .../interop/InteropBootstrapFactory.java        |   39 +
 .../internal/interop/InteropIgnition.java       |  166 +
 .../internal/interop/InteropProcessor.java      |   36 +
 .../communication/GridIoMessageFactory.java     |    4 +-
 .../eventstorage/GridEventStorageManager.java   |    5 +-
 .../cache/DynamicCacheDescriptor.java           |   16 +-
 .../processors/cache/GridCacheAdapter.java      |  581 +-
 .../processors/cache/GridCacheContext.java      |    7 +
 .../cache/GridCacheEvictionManager.java         |    2 +-
 .../processors/cache/GridCacheMapEntry.java     |  123 +-
 .../processors/cache/GridCacheMvccManager.java  |    4 +-
 .../GridCachePartitionExchangeManager.java      |    3 +
 .../processors/cache/GridCacheProcessor.java    |  189 +-
 .../processors/cache/GridCacheProxyImpl.java    |   24 -
 .../processors/cache/GridCacheSwapManager.java  |  215 +-
 .../processors/cache/GridCacheTtlManager.java   |   42 +-
 .../processors/cache/GridCacheUtils.java        |    5 +-
 .../processors/cache/IgniteInternalCache.java   |   27 -
 ...ridCacheOptimisticCheckPreparedTxFuture.java |  434 --
 ...idCacheOptimisticCheckPreparedTxRequest.java |  232 -
 ...dCacheOptimisticCheckPreparedTxResponse.java |  179 -
 .../distributed/GridCacheTxRecoveryFuture.java  |  506 ++
 .../distributed/GridCacheTxRecoveryRequest.java |  261 +
 .../GridCacheTxRecoveryResponse.java            |  182 +
 .../GridDistributedCacheAdapter.java            |  210 +-
 .../distributed/GridDistributedLockRequest.java |  111 +-
 .../GridDistributedTxFinishRequest.java         |   70 +-
 .../GridDistributedTxPrepareRequest.java        |  112 +-
 .../GridDistributedTxRemoteAdapter.java         |   22 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   16 +-
 .../distributed/dht/GridDhtLocalPartition.java  |    2 +-
 .../distributed/dht/GridDhtLockFuture.java      |    2 -
 .../distributed/dht/GridDhtLockRequest.java     |   45 +-
 .../dht/GridDhtOffHeapCacheEntry.java           |   63 +
 .../dht/GridDhtTransactionalCacheAdapter.java   |   15 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |    3 -
 .../distributed/dht/GridDhtTxFinishRequest.java |   43 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   38 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   95 +-
 .../cache/distributed/dht/GridDhtTxMapping.java |    2 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   99 +-
 .../dht/GridDhtTxPrepareRequest.java            |   60 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |    8 +-
 .../distributed/dht/GridNoStorageCacheMap.java  |    4 +-
 .../dht/GridPartitionedGetFuture.java           |    2 +-
 .../dht/atomic/GridDhtAtomicCache.java          |    3 +
 .../atomic/GridDhtAtomicOffHeapCacheEntry.java  |   63 +
 .../dht/colocated/GridDhtColocatedCache.java    |    5 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   33 +-
 .../GridDhtColocatedOffHeapCacheEntry.java      |   63 +
 .../colocated/GridDhtDetachedCacheEntry.java    |    4 +-
 .../GridDhtPartitionsExchangeFuture.java        |    2 +-
 .../distributed/near/GridNearCacheAdapter.java  |   13 +-
 .../distributed/near/GridNearCacheEntry.java    |    4 +-
 .../distributed/near/GridNearLockFuture.java    |   11 -
 .../distributed/near/GridNearLockRequest.java   |   61 +-
 .../near/GridNearOffHeapCacheEntry.java         |   60 +
 .../near/GridNearOptimisticTxPrepareFuture.java |  768 ++
 .../GridNearPessimisticTxPrepareFuture.java     |  347 +
 .../near/GridNearTransactionalCache.java        |    4 -
 .../near/GridNearTxFinishRequest.java           |   28 +-
 .../cache/distributed/near/GridNearTxLocal.java |  104 +-
 .../near/GridNearTxPrepareFuture.java           | 1050 ---
 .../near/GridNearTxPrepareFutureAdapter.java    |  226 +
 .../near/GridNearTxPrepareRequest.java          |   52 +-
 .../distributed/near/GridNearTxRemote.java      |   24 +-
 .../processors/cache/local/GridLocalCache.java  |    8 +-
 .../cache/local/GridLocalCacheEntry.java        |   18 +
 .../local/atomic/GridLocalAtomicCache.java      |   27 +-
 .../cache/query/GridCacheQueryManager.java      |   21 +-
 .../cache/query/GridCacheSqlQuery.java          |    2 +-
 .../cache/query/GridCacheTwoStepQuery.java      |   17 +
 .../cache/transactions/IgniteInternalTx.java    |   19 +-
 .../transactions/IgniteTransactionsImpl.java    |    4 +-
 .../cache/transactions/IgniteTxAdapter.java     |   76 +-
 .../cache/transactions/IgniteTxEntry.java       |   48 +-
 .../cache/transactions/IgniteTxHandler.java     |  106 +-
 .../transactions/IgniteTxLocalAdapter.java      |  181 +-
 .../cache/transactions/IgniteTxLocalEx.java     |   21 +-
 .../cache/transactions/IgniteTxManager.java     |  247 +-
 .../datastreamer/DataStreamerImpl.java          |    2 +
 .../processors/igfs/IgfsDataManager.java        |    3 +
 .../processors/igfs/IgfsDeleteWorker.java       |    4 +
 .../processors/igfs/IgfsMetaManager.java        |    2 +-
 .../internal/processors/igfs/IgfsUtils.java     |   11 +-
 .../offheap/GridOffHeapProcessor.java           |   17 +
 .../processors/resource/GridResourceField.java  |   16 +-
 .../processors/resource/GridResourceIoc.java    |  389 +-
 .../processors/resource/GridResourceMethod.java |   13 +
 .../resource/GridResourceProcessor.java         |   20 +-
 .../ignite/internal/util/IgniteUtils.java       |   22 +-
 .../util/lang/GridComputeJobWrapper.java        |   96 -
 .../ignite/internal/util/lang/GridFunc.java     | 7218 +++++-------------
 .../internal/util/nio/GridBufferedParser.java   |    4 -
 .../internal/util/nio/GridDelimitedParser.java  |   91 +
 .../util/nio/GridNioDelimitedBuffer.java        |  106 +
 .../util/offheap/GridOffHeapPartitionedMap.java |    9 +
 .../unsafe/GridUnsafePartitionedMap.java        |  155 +-
 .../internal/visor/query/VisorQueryArg.java     |   14 +-
 .../internal/visor/query/VisorQueryJob.java     |    2 +
 .../communication/tcp/TcpCommunicationSpi.java  |    2 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   48 +-
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |   10 +-
 .../org/apache/ignite/stream/StreamAdapter.java |  111 +
 .../ignite/stream/StreamTupleExtractor.java     |   33 +
 .../stream/socket/SocketMessageConverter.java   |   31 +
 .../ignite/stream/socket/SocketStreamer.java    |  218 +
 .../ignite/stream/socket/package-info.java      |   21 +
 .../resources/META-INF/classnames.properties    |   13 +-
 .../core/src/main/resources/ignite.properties   |    2 +-
 .../internal/GridUpdateNotifierSelfTest.java    |   21 +-
 .../processors/cache/CacheGetFromJobTest.java   |  110 +
 .../cache/CacheOffheapMapEntrySelfTest.java     |  168 +
 .../cache/CacheRemoveAllSelfTest.java           |   81 +
 .../GridCacheAbstractFailoverSelfTest.java      |   12 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  227 +-
 .../cache/GridCacheAbstractSelfTest.java        |    4 +-
 .../cache/OffHeapTieredTransactionSelfTest.java |  127 +
 .../GridCacheAbstractNodeRestartSelfTest.java   |  101 +-
 .../distributed/GridCacheLockAbstractTest.java  |    2 -
 .../distributed/IgniteTxGetAfterStopTest.java   |  131 +
 ...xOriginatingNodeFailureAbstractSelfTest.java |    2 +-
 ...icOffHeapTieredMultiNodeFullApiSelfTest.java |   43 +
 ...achePartitionedNearDisabledLockSelfTest.java |   47 +
 ...ionedNearDisabledOffHeapFullApiSelfTest.java |    8 +-
 ...DisabledOffHeapMultiNodeFullApiSelfTest.java |    8 +-
 ...abledOffHeapTieredAtomicFullApiSelfTest.java |   56 +
 ...earDisabledOffHeapTieredFullApiSelfTest.java |   33 +
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |   33 +
 ...rDisabledPrimaryNodeFailureRecoveryTest.java |   31 +
 ...rtitionedPrimaryNodeFailureRecoveryTest.java |   31 +
 ...woBackupsPrimaryNodeFailureRecoveryTest.java |   37 +
 ...ePrimaryNodeFailureRecoveryAbstractTest.java |  533 ++
 ...idCacheAtomicReplicatedFailoverSelfTest.java |    6 +
 ...CacheAtomicOffHeapTieredFullApiSelfTest.java |   32 +
 ...icOffHeapTieredMultiNodeFullApiSelfTest.java |   33 +
 ...yWriteOrderOffHeapTieredFullApiSelfTest.java |   33 +
 ...erOffHeapTieredMultiNodeFullApiSelfTest.java |   33 +
 ...achePartitionedMultiNodeFullApiSelfTest.java |   15 +-
 .../GridCachePartitionedNodeRestartTest.java    |    4 +-
 ...dCachePartitionedOffHeapFullApiSelfTest.java |    8 +-
 ...titionedOffHeapMultiNodeFullApiSelfTest.java |    8 +-
 ...PartitionedOffHeapTieredFullApiSelfTest.java |   32 +
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |   72 +
 ...ePartitionedOptimisticTxNodeRestartTest.java |    4 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |   25 +-
 .../near/IgniteCacheNearOnlyTxTest.java         |  190 +
 .../near/NoneRebalanceModeSelfTest.java         |   67 +
 .../GridCacheReplicatedFailoverSelfTest.java    |    6 +
 .../GridCacheReplicatedLockSelfTest.java        |    5 +
 .../GridCacheReplicatedNodeRestartSelfTest.java |   82 +
 ...idCacheReplicatedOffHeapFullApiSelfTest.java |    8 +-
 ...plicatedOffHeapMultiNodeFullApiSelfTest.java |    8 +-
 ...eReplicatedOffHeapTieredFullApiSelfTest.java |   33 +
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |   33 +
 .../IgniteCacheExpiryPolicyAbstractTest.java    |    2 +-
 .../IgniteCacheExpiryPolicyTestSuite.java       |    2 +
 .../expiry/IgniteCacheTtlCleanupSelfTest.java   |   85 +
 ...LocalAtomicOffHeapTieredFullApiSelfTest.java |   32 +
 .../GridCacheLocalIsolatedNodesSelfTest.java    |   18 +-
 .../GridCacheLocalOffHeapFullApiSelfTest.java   |    6 +-
 ...dCacheLocalOffHeapTieredFullApiSelfTest.java |   32 +
 .../igfs/IgfsClientCacheSelfTest.java           |  132 +
 .../processors/igfs/IgfsOneClientNodeTest.java  |  133 +
 .../processors/igfs/IgfsStreamsSelfTest.java    |    2 +-
 .../util/nio/GridNioDelimitedBufferTest.java    |  112 +
 .../discovery/tcp/TcpDiscoveryRestartTest.java  |  199 +
 .../stream/socket/SocketStreamerSelfTest.java   |  315 +
 .../ignite/stream/socket/package-info.java      |   21 +
 .../ignite/testsuites/IgniteBasicTestSuite.java |    1 +
 .../IgniteCacheFailoverTestSuite.java           |   10 +-
 .../IgniteCacheFullApiSelfTestSuite.java        |   18 +
 .../testsuites/IgniteCacheRestartTestSuite.java |   11 +-
 .../testsuites/IgniteCacheTestSuite2.java       |    3 +
 .../testsuites/IgniteCacheTestSuite3.java       |    5 +-
 .../testsuites/IgniteCacheTestSuite4.java       |    4 +
 .../IgniteCacheTxRecoverySelfTestSuite.java     |    4 +
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |    3 +
 .../testsuites/IgniteStreamTestSuite.java       |   39 +
 .../testsuites/IgniteUtilSelfTestSuite.java     |    1 +
 modules/extdata/p2p/pom.xml                     |    2 +-
 modules/extdata/uri/pom.xml                     |    2 +-
 modules/gce/pom.xml                             |    4 +-
 modules/geospatial/pom.xml                      |    2 +-
 modules/hadoop/pom.xml                          |    2 +-
 modules/hibernate/pom.xml                       |    2 +-
 modules/indexing/pom.xml                        |    2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |    4 +
 .../processors/query/h2/sql/GridSqlQuery.java   |   20 +
 .../query/h2/sql/GridSqlQueryParser.java        |   10 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |   11 +-
 .../processors/query/h2/sql/GridSqlSelect.java  |    2 +-
 .../processors/query/h2/sql/GridSqlUnion.java   |    2 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |    3 +
 .../h2/twostep/GridReduceQueryExecutor.java     |  119 +-
 .../cache/GridCacheOffheapIndexGetSelfTest.java |  111 +
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |   21 +
 ...eQueryMultiThreadedOffHeapTiredSelfTest.java |   37 +
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   29 +-
 .../IgniteCacheQuerySelfTestSuite.java          |    1 +
 .../IgniteCacheWithIndexingTestSuite.java       |    2 +
 modules/jcl/pom.xml                             |    2 +-
 modules/jta/pom.xml                             |    2 +-
 .../processors/cache/jta/CacheJtaManager.java   |    4 +-
 modules/log4j/pom.xml                           |    2 +-
 modules/rest-http/pom.xml                       |    2 +-
 modules/scalar/pom.xml                          |    2 +-
 .../ignite/scalar/ScalarConversions.scala       |    8 -
 .../scalar/tests/ScalarCacheQueriesSpec.scala   |  154 +-
 .../ignite/scalar/tests/ScalarCacheSpec.scala   |   23 +-
 .../scalar/tests/ScalarConversionsSpec.scala    |   43 +-
 .../scalar/tests/ScalarProjectionSpec.scala     |  128 +-
 .../scalar/tests/ScalarReturnableSpec.scala     |   41 +-
 modules/schedule/pom.xml                        |    2 +-
 modules/schema-import/pom.xml                   |    2 +-
 .../ignite/schema/generator/CodeGenerator.java  |   41 +-
 modules/slf4j/pom.xml                           |    2 +-
 modules/spring/pom.xml                          |    2 +-
 modules/ssh/pom.xml                             |    2 +-
 modules/tools/pom.xml                           |    2 +-
 modules/urideploy/pom.xml                       |    2 +-
 modules/visor-console/pom.xml                   |    4 +-
 .../commands/cache/VisorCacheScanCommand.scala  |    2 +-
 .../ignite/visor/VisorRuntimeBaseSpec.scala     |    2 +-
 .../visor/commands/VisorArgListSpec.scala       |   60 +-
 .../commands/VisorFileNameCompleterSpec.scala   |   34 +-
 .../commands/ack/VisorAckCommandSpec.scala      |   20 +-
 .../commands/alert/VisorAlertCommandSpec.scala  |   68 +-
 .../cache/VisorCacheClearCommandSpec.scala      |   48 +-
 .../commands/cache/VisorCacheCommandSpec.scala  |   66 +-
 .../config/VisorConfigurationCommandSpec.scala  |    8 +-
 .../cswap/VisorCacheSwapCommandSpec.scala       |   24 +-
 .../deploy/VisorDeployCommandSpec.scala         |   10 +-
 .../disco/VisorDiscoveryCommandSpec.scala       |   46 +-
 .../events/VisorEventsCommandSpec.scala         |   28 +-
 .../visor/commands/gc/VisorGcCommandSpec.scala  |   30 +-
 .../commands/help/VisorHelpCommandSpec.scala    |   57 +-
 .../commands/kill/VisorKillCommandSpec.scala    |   58 +-
 .../commands/log/VisorLogCommandSpec.scala      |   10 +-
 .../commands/mem/VisorMemoryCommandSpec.scala   |   77 +-
 .../commands/node/VisorNodeCommandSpec.scala    |   22 +-
 .../commands/open/VisorOpenCommandSpec.scala    |   16 +-
 .../commands/ping/VisorPingCommandSpec.scala    |   16 +-
 .../commands/start/VisorStartCommandSpec.scala  |  126 +-
 .../commands/tasks/VisorTasksCommandSpec.scala  |  112 +-
 .../commands/top/VisorTopologyCommandSpec.scala |   52 +-
 .../commands/vvm/VisorVvmCommandSpec.scala      |   30 +-
 modules/visor-plugins/pom.xml                   |    2 +-
 modules/web/pom.xml                             |    2 +-
 modules/yardstick/pom.xml                       |    2 +-
 parent/pom.xml                                  |    2 +
 pom.xml                                         |  117 +-
 286 files changed, 12125 insertions(+), 11071 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fd6f3989/pom.xml
----------------------------------------------------------------------


[31/37] incubator-ignite git commit: [IGNITE-765]: org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint.GcWorker#cleanupResources incorrectly handles FileLockInterruptionException

Posted by sb...@apache.org.
[IGNITE-765]: org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint.GcWorker#cleanupResources incorrectly handles FileLockInterruptionException


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

Branch: refs/heads/ignite-gg-10369
Commit: 98e392cd79183945c26e9c220787cdbd4f686c26
Parents: b6fc8a9
Author: iveselovskiy <iv...@gridgain.com>
Authored: Thu May 28 18:39:45 2015 +0300
Committer: iveselovskiy <iv...@gridgain.com>
Committed: Thu May 28 18:39:45 2015 +0300

----------------------------------------------------------------------
 .../internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/98e392cd/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
index 86a0886..5185856 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
@@ -592,7 +592,7 @@ public class IpcSharedMemoryServerEndpoint implements IpcServerEndpoint {
                 if (log.isDebugEnabled())
                     log.debug("Token directory is being processed concurrently: " + workTokDir.getAbsolutePath());
             }
-            catch (InterruptedIOException ignored) {
+            catch (FileLockInterruptionException ignored) {
                 Thread.currentThread().interrupt();
             }
             catch (IOException e) {


[11/37] incubator-ignite git commit: #IGNITE-857 Update javadoc.

Posted by sb...@apache.org.
#IGNITE-857 Update javadoc.


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

Branch: refs/heads/ignite-gg-10369
Commit: c462a0f8d09e292e60c342139bf34513a7a1db40
Parents: 76a0d0a
Author: nikolay tikhonov <nt...@gridgain.com>
Authored: Mon May 25 19:04:49 2015 +0300
Committer: nikolay tikhonov <nt...@gridgain.com>
Committed: Mon May 25 19:04:49 2015 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/mesos/ClusterProperties.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c462a0f8/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
index b3093f8..63ef27c 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
@@ -125,7 +125,7 @@ public class ClusterProperties {
     public static final String IGNITE_WORK_DIR = "IGNITE_WORK_DIR";
 
     /** */
-    public static final String DEFAULT_IGNITE_WORK_DIR = "~/ignite-releases";
+    public static final String DEFAULT_IGNITE_WORK_DIR = "~/ignite-releases/";
 
     /** Ignite version. */
     private String igniteWorkDir = DEFAULT_IGNITE_WORK_DIR;


[15/37] incubator-ignite git commit: #IGNITE-857 WIP

Posted by sb...@apache.org.
#IGNITE-857 WIP


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

Branch: refs/heads/ignite-gg-10369
Commit: 29710de2bb1d09357b502f2dda16387c89c1bacc
Parents: 639c912
Author: nikolay tikhonov <nt...@gridgain.com>
Authored: Tue May 26 16:38:20 2015 +0300
Committer: nikolay tikhonov <nt...@gridgain.com>
Committed: Tue May 26 16:38:20 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/mesos/ClusterProperties.java  | 81 +++++++++++++++-----
 .../apache/ignite/mesos/IgniteScheduler.java    | 42 +++++++---
 2 files changed, 92 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/29710de2/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
index de0afcf..e659509 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
@@ -53,49 +53,49 @@ public class ClusterProperties {
     private int httpServerPort = Integer.valueOf(DEFAULT_HTTP_SERVER_PORT);
 
     /** */
-    public static final String IGNITE_RESOURCE_CPU_CORES = "IGNITE_RESOURCE_CPU_CORES";
+    public static final String IGNITE_TOTAL_CPU = "IGNITE_TOTAL_CPU";
 
     /** CPU limit. */
     private double cpu = UNLIMITED;
 
     /** */
-    public static final String IGNITE_RESOURCE_CPU_CORES_PER_NODE = "IGNITE_RESOURCE_CPU_CORES_PER_NODE";
+    public static final String IGNITE_RUN_CPU_PER_NODE = "IGNITE_RUN_CPU_PER_NODE";
 
     /** CPU limit. */
     private double cpuPerNode = UNLIMITED;
 
     /** */
-    public static final String IGNITE_RESOURCE_MEM_MB = "IGNITE_RESOURCE_MEM_MB";
+    public static final String IGNITE_TOTAL_MEMORY = "IGNITE_TOTAL_MEMORY";
 
     /** Memory limit. */
     private double mem = UNLIMITED;
 
     /** */
-    public static final String IGNITE_RESOURCE_MEM_MB_PER_NODE = "IGNITE_RESOURCE_MEM_MB_PER_NODE";
+    public static final String IGNITE_MEMORY_PER_NODE = "IGNITE_MEMORY_PER_NODE";
 
     /** Memory limit. */
     private double memPerNode = UNLIMITED;
 
     /** */
-    public static final String IGNITE_RESOURCE_DISK_MB = "IGNITE_RESOURCE_DISK_MB";
+    public static final String IGNITE_TOTAL_DISK_SPACE = "IGNITE_TOTAL_DISK_SPACE";
 
     /** Disk space limit. */
     private double disk = UNLIMITED;
 
     /** */
-    public static final String IGNITE_RESOURCE_DISK_MB_PER_NODE = "IGNITE_RESOURCE_DISK_MB_PER_NODE";
+    public static final String IGNITE_DISK_SPACE_PER_NODE = "IGNITE_DISK_SPACE_PER_NODE";
 
     /** Disk space limit. */
     private double diskPerNode = UNLIMITED;
 
     /** */
-    public static final String IGNITE_RESOURCE_NODE_CNT = "IGNITE_RESOURCE_NODE_CNT";
+    public static final String IGNITE_NODE_COUNT = "IGNITE_NODE_COUNT";
 
     /** Node count limit. */
     private double nodeCnt = UNLIMITED;
 
     /** */
-    public static final String IGNITE_RESOURCE_MIN_CPU_CNT_PER_NODE = "IGNITE_RESOURCE_MIN_CPU_CNT_PER_NODE";
+    public static final String IGNITE_MIN_CPU_PER_NODE = "IGNITE_MIN_CPU_PER_NODE";
 
     /** */
     public static final double DEFAULT_RESOURCE_MIN_CPU = 1;
@@ -104,7 +104,7 @@ public class ClusterProperties {
     private double minCpu = DEFAULT_RESOURCE_MIN_CPU;
 
     /** */
-    public static final String IGNITE_RESOURCE_MIN_MEMORY_PER_NODE = "IGNITE_RESOURCE_MIN_MEMORY_PER_NODE";
+    public static final String IGNITE_MIN_MEMORY_PER_NODE = "IGNITE_MIN_MEMORY_PER_NODE";
 
     /** */
     public static final double DEFAULT_RESOURCE_MIN_MEM = 256;
@@ -122,10 +122,16 @@ public class ClusterProperties {
     private String igniteVer = DEFAULT_IGNITE_VERSION;
 
     /** */
+    public static final String IGNITE_PACKAGE_URL = "IGNITE_PACKAGE_URL";
+
+    /** Ignite package url. */
+    private String ignitePackageUrl = null;
+
+    /** */
     public static final String IGNITE_WORK_DIR = "IGNITE_WORK_DIR";
 
     /** */
-    public static final String DEFAULT_IGNITE_WORK_DIR = "~/ignite-releases/";
+    public static final String DEFAULT_IGNITE_WORK_DIR = "ignite-releases/";
 
     /** Ignite version. */
     private String igniteWorkDir = DEFAULT_IGNITE_WORK_DIR;
@@ -137,12 +143,24 @@ public class ClusterProperties {
     private String userLibs = null;
 
     /** */
+    public static final String IGNITE_USERS_LIBS_URL = "IGNITE_USERS_LIBS_URL";
+
+    /** URL to users libs. */
+    private String userLibsUrl = null;
+
+    /** */
     public static final String IGNITE_CONFIG_XML = "IGNITE_XML_CONFIG";
 
     /** Ignite config. */
     private String igniteCfg = null;
 
     /** */
+    public static final String IGNITE_CONFIG_XML_URL = "IGNITE_XML_CONFIG_URL";
+
+    /** Url to ignite config. */
+    private String igniteCfgUrl = null;
+
+    /** */
     public ClusterProperties() {
         // No-op.
     }
@@ -293,6 +311,27 @@ public class ClusterProperties {
     }
 
     /**
+     * @return Url to ignite package.
+     */
+    public String ignitePackageUrl() {
+        return ignitePackageUrl;
+    }
+
+    /**
+     * @return Url to ignite configuration.
+     */
+    public String igniteConfigUrl() {
+        return igniteCfgUrl;
+    }
+
+    /**
+     * @return Url to users libs configuration.
+     */
+    public String usersLibsUrl() {
+        return userLibsUrl;
+    }
+
+    /**
      * @param config path to config file.
      * @return Cluster configuration.
      */
@@ -314,15 +353,19 @@ public class ClusterProperties {
             prop.httpServerPort = Integer.valueOf(getStringProperty(IGNITE_HTTP_SERVER_PORT, props,
                 DEFAULT_HTTP_SERVER_PORT));
 
-            prop.cpu = getDoubleProperty(IGNITE_RESOURCE_CPU_CORES, props, UNLIMITED);
-            prop.cpuPerNode = getDoubleProperty(IGNITE_RESOURCE_CPU_CORES_PER_NODE, props, UNLIMITED);
-            prop.mem = getDoubleProperty(IGNITE_RESOURCE_MEM_MB, props, UNLIMITED);
-            prop.memPerNode = getDoubleProperty(IGNITE_RESOURCE_MEM_MB_PER_NODE, props, UNLIMITED);
-            prop.disk = getDoubleProperty(IGNITE_RESOURCE_DISK_MB, props, UNLIMITED);
-            prop.diskPerNode = getDoubleProperty(IGNITE_RESOURCE_DISK_MB_PER_NODE, props, 1024.0);
-            prop.nodeCnt = getDoubleProperty(IGNITE_RESOURCE_NODE_CNT, props, UNLIMITED);
-            prop.minCpu = getDoubleProperty(IGNITE_RESOURCE_MIN_CPU_CNT_PER_NODE, props, DEFAULT_RESOURCE_MIN_CPU);
-            prop.minMemory = getDoubleProperty(IGNITE_RESOURCE_MIN_MEMORY_PER_NODE, props, DEFAULT_RESOURCE_MIN_MEM);
+            prop.userLibsUrl = getStringProperty(IGNITE_USERS_LIBS_URL, props, null);
+            prop.ignitePackageUrl = getStringProperty(IGNITE_PACKAGE_URL, props, null);
+            prop.igniteCfgUrl = getStringProperty(IGNITE_CONFIG_XML_URL, props, null);
+
+            prop.cpu = getDoubleProperty(IGNITE_TOTAL_CPU, props, UNLIMITED);
+            prop.cpuPerNode = getDoubleProperty(IGNITE_RUN_CPU_PER_NODE, props, UNLIMITED);
+            prop.mem = getDoubleProperty(IGNITE_TOTAL_MEMORY, props, UNLIMITED);
+            prop.memPerNode = getDoubleProperty(IGNITE_MEMORY_PER_NODE, props, UNLIMITED);
+            prop.disk = getDoubleProperty(IGNITE_TOTAL_DISK_SPACE, props, UNLIMITED);
+            prop.diskPerNode = getDoubleProperty(IGNITE_DISK_SPACE_PER_NODE, props, 1024.0);
+            prop.nodeCnt = getDoubleProperty(IGNITE_NODE_COUNT, props, UNLIMITED);
+            prop.minCpu = getDoubleProperty(IGNITE_MIN_CPU_PER_NODE, props, DEFAULT_RESOURCE_MIN_CPU);
+            prop.minMemory = getDoubleProperty(IGNITE_MIN_MEMORY_PER_NODE, props, DEFAULT_RESOURCE_MIN_MEM);
 
             prop.igniteVer = getStringProperty(IGNITE_VERSION, props, DEFAULT_IGNITE_VERSION);
             prop.igniteWorkDir = getStringProperty(IGNITE_WORK_DIR, props, DEFAULT_IGNITE_WORK_DIR);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/29710de2/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index 6b165e4..ad48189 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -117,19 +117,37 @@ public class IgniteScheduler implements Scheduler {
                 .setName("IGNITE_TCP_DISCOVERY_ADDRESSES")
                 .setValue(getAddress(offer.getHostname()))))
             .addUris(Protos.CommandInfo.URI.newBuilder()
-                .setValue(resourceProvider.igniteUrl())
+                .setValue(clusterProps.ignitePackageUrl() != null ?
+                    clusterProps.ignitePackageUrl() : resourceProvider.igniteUrl())
                 .setExtract(true))
-            .addUris(Protos.CommandInfo.URI.newBuilder().setValue(resourceProvider.igniteConfigUrl()));
-
-        if (resourceProvider.resourceUrl() != null) {
-            for (String url : resourceProvider.resourceUrl())
-                builder.addUris(Protos.CommandInfo.URI.newBuilder().setValue(url));
-
-            builder.setValue("cp *.jar ./gridgain-community-*/libs/ "
-                + "&& ./gridgain-community-*/bin/ignite.sh "
-                + resourceProvider.configName()
-                + " -J-Xmx" + String.valueOf((int)igniteTask.mem() + "m")
-                + " -J-Xms" + String.valueOf((int)igniteTask.mem()) + "m");
+            .addUris(Protos.CommandInfo.URI.newBuilder()
+                .setValue(clusterProps.igniteConfigUrl() != null ?
+                    clusterProps.igniteConfigUrl() : resourceProvider.igniteConfigUrl()));
+
+        if (resourceProvider.resourceUrl() != null || clusterProps.usersLibsUrl() != null) {
+            if (clusterProps.igniteConfigUrl() != null) {
+                builder.addUris(Protos.CommandInfo.URI.newBuilder()
+                    .setValue(clusterProps.igniteConfigUrl())
+                    .setExtract(true));
+
+                String[] split = clusterProps.igniteConfigUrl().split("/");
+
+                builder.setValue("cp *.jar ./gridgain-community-*/libs/ "
+                    + "&& ./gridgain-community-*/bin/ignite.sh "
+                    + split[split.length - 1]
+                    + " -J-Xmx" + String.valueOf((int)igniteTask.mem() + "m")
+                    + " -J-Xms" + String.valueOf((int)igniteTask.mem()) + "m");
+            }
+            else {
+                for (String url : resourceProvider.resourceUrl())
+                    builder.addUris(Protos.CommandInfo.URI.newBuilder().setValue(url));
+
+                builder.setValue("cp *.jar ./gridgain-community-*/libs/ "
+                        + "&& ./gridgain-community-*/bin/ignite.sh "
+                        + resourceProvider.configName()
+                        + " -J-Xmx" + String.valueOf((int)igniteTask.mem() + "m")
+                        + " -J-Xms" + String.valueOf((int)igniteTask.mem()) + "m");
+            }
         }
         else
             builder.setValue("./gridgain-community-*/bin/ignite.sh "


[25/37] incubator-ignite git commit: Merge branches 'ignite-857' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-857

Posted by sb...@apache.org.
Merge branches 'ignite-857' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-857


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

Branch: refs/heads/ignite-gg-10369
Commit: 789106b634b339630da31da74b56f72ac3ac14fc
Parents: 4884995 b6fc8a9
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu May 28 17:08:00 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu May 28 17:08:00 2015 +0300

----------------------------------------------------------------------
 dev-tools/gradle/wrapper/gradle-wrapper.jar     | Bin 51017 -> 0 bytes
 .../gradle/wrapper/gradle-wrapper.properties    |  18 +-
 dev-tools/gradlew                               | 163 ++++++++----
 dev-tools/slurp.sh                              |   2 +-
 dev-tools/src/main/groovy/jiraslurp.groovy      |  73 ++++--
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../src/main/java/org/apache/ignite/Ignite.java |   8 +-
 .../java/org/apache/ignite/IgniteServices.java  |   5 +-
 .../apache/ignite/internal/IgniteKernal.java    |   4 +-
 .../org/apache/ignite/services/Service.java     |   5 +-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  20 +-
 .../service/ClosureServiceClientsNodesTest.java | 245 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 modules/extdata/p2p/pom.xml                     |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |   4 +
 .../query/h2/sql/BaseH2CompareQueryTest.java    |  16 ++
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 .../licenses/jcraft-revised-bsd.txt             |  28 ---
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |   2 +-
 47 files changed, 498 insertions(+), 159 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/789106b6/pom.xml
----------------------------------------------------------------------


[10/37] incubator-ignite git commit: #IGNITE-857 Update javadoc.

Posted by sb...@apache.org.
#IGNITE-857 Update javadoc.


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

Branch: refs/heads/ignite-gg-10369
Commit: 76a0d0aeb676366a2733b25d8162bdfeb039354d
Parents: 4b48218
Author: nikolay tikhonov <nt...@gridgain.com>
Authored: Mon May 25 19:03:49 2015 +0300
Committer: nikolay tikhonov <nt...@gridgain.com>
Committed: Mon May 25 19:03:49 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/mesos/ClusterProperties.java  | 106 ++++++++++++++++++-
 .../apache/ignite/mesos/IgniteFramework.java    |  43 +++-----
 .../apache/ignite/mesos/IgniteScheduler.java    |  26 ++---
 .../org/apache/ignite/mesos/IgniteTask.java     |   2 +-
 .../org/apache/ignite/mesos/package-info.java   |   2 +-
 .../ignite/mesos/resource/IgniteProvider.java   |   2 +-
 .../mesos/resource/ResourceController.java      |  25 ++---
 .../ignite/mesos/resource/ResourceProvider.java |   2 +-
 .../ignite/mesos/resource/package-info.java     |  22 ++++
 9 files changed, 166 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/76a0d0ae/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
index bb7f7a4..b3093f8 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.mesos;
 
 import java.io.*;
+import java.net.*;
 import java.util.*;
 
 /**
@@ -25,7 +26,7 @@ import java.util.*;
  */
 public class ClusterProperties {
     /** Unlimited. */
-    public static final double UNLIMITED = -1;
+    public static final double UNLIMITED = Double.MAX_VALUE;
 
     /** */
     public static final String MESOS_MASTER_URL = "MESOS_MASTER_URL";
@@ -37,24 +38,57 @@ public class ClusterProperties {
     private String mesosUrl = DEFAULT_MESOS_MASTER_URL;
 
     /** */
+    public static final String IGNITE_HTTP_SERVER_HOST = "IGNITE_HTTP_SERVER_HOST";
+
+    /** Http server host. */
+    private String httpServerHost = null;
+
+    /** */
+    public static final String IGNITE_HTTP_SERVER_PORT = "IGNITE_HTTP_SERVER_PORT";
+
+    /** */
+    public static final String DEFAULT_HTTP_SERVER_PORT = "47511";
+
+    /** Http server host. */
+    private int httpServerPort = Integer.valueOf(DEFAULT_HTTP_SERVER_PORT);
+
+    /** */
     public static final String IGNITE_RESOURCE_CPU_CORES = "IGNITE_RESOURCE_CPU_CORES";
 
     /** CPU limit. */
     private double cpu = UNLIMITED;
 
     /** */
+    public static final String IGNITE_RESOURCE_CPU_CORES_PER_NODE = "IGNITE_RESOURCE_CPU_CORES_PER_NODE";
+
+    /** CPU limit. */
+    private double cpuPerNode = UNLIMITED;
+
+    /** */
     public static final String IGNITE_RESOURCE_MEM_MB = "IGNITE_RESOURCE_MEM_MB";
 
     /** Memory limit. */
     private double mem = UNLIMITED;
 
     /** */
+    public static final String IGNITE_RESOURCE_MEM_MB_PER_NODE = "IGNITE_RESOURCE_MEM_MB_PER_NODE";
+
+    /** Memory limit. */
+    private double memPerNode = UNLIMITED;
+
+    /** */
     public static final String IGNITE_RESOURCE_DISK_MB = "IGNITE_RESOURCE_DISK_MB";
 
     /** Disk space limit. */
     private double disk = UNLIMITED;
 
     /** */
+    public static final String IGNITE_RESOURCE_DISK_MB_PER_NODE = "IGNITE_RESOURCE_DISK_MB_PER_NODE";
+
+    /** Disk space limit. */
+    private double diskPerNode = UNLIMITED;
+
+    /** */
     public static final String IGNITE_RESOURCE_NODE_CNT = "IGNITE_RESOURCE_NODE_CNT";
 
     /** Node count limit. */
@@ -121,6 +155,13 @@ public class ClusterProperties {
     }
 
     /**
+     * @return CPU count limit.
+     */
+    public double cpusPerNode(){
+        return cpuPerNode;
+    }
+
+    /**
      * @return mem limit.
      */
     public double memory() {
@@ -128,6 +169,13 @@ public class ClusterProperties {
     }
 
     /**
+     * @return mem limit.
+     */
+    public double memoryPerNode() {
+        return memPerNode;
+    }
+
+    /**
      * @return disk limit.
      */
     public double disk() {
@@ -135,6 +183,13 @@ public class ClusterProperties {
     }
 
     /**
+     * @return disk limit per node.
+     */
+    public double diskPerNode() {
+        return diskPerNode;
+    }
+
+    /**
      * @return instance count limit.
      */
     public double instances() {
@@ -191,6 +246,20 @@ public class ClusterProperties {
     }
 
     /**
+     * @return Http server host.
+     */
+    public String httpServerHost() {
+        return httpServerHost;
+    }
+
+    /**
+     * @return Http server port.
+     */
+    public int httpServerPort() {
+        return httpServerPort;
+    }
+
+    /**
      * @param config path to config file.
      * @return Cluster configuration.
      */
@@ -208,9 +277,16 @@ public class ClusterProperties {
 
             prop.mesosUrl = getStringProperty(MESOS_MASTER_URL, props, DEFAULT_MESOS_MASTER_URL);
 
+            prop.httpServerHost = getStringProperty(IGNITE_HTTP_SERVER_HOST, props, getNonLoopbackAddress());
+            prop.httpServerPort = Integer.valueOf(getStringProperty(IGNITE_HTTP_SERVER_PORT, props,
+                DEFAULT_HTTP_SERVER_PORT));
+
             prop.cpu = getDoubleProperty(IGNITE_RESOURCE_CPU_CORES, props, UNLIMITED);
+            prop.cpuPerNode = getDoubleProperty(IGNITE_RESOURCE_CPU_CORES_PER_NODE, props, UNLIMITED);
             prop.mem = getDoubleProperty(IGNITE_RESOURCE_MEM_MB, props, UNLIMITED);
+            prop.memPerNode = getDoubleProperty(IGNITE_RESOURCE_MEM_MB_PER_NODE, props, UNLIMITED);
             prop.disk = getDoubleProperty(IGNITE_RESOURCE_DISK_MB, props, UNLIMITED);
+            prop.diskPerNode = getDoubleProperty(IGNITE_RESOURCE_DISK_MB_PER_NODE, props, UNLIMITED);
             prop.nodeCnt = getDoubleProperty(IGNITE_RESOURCE_NODE_CNT, props, UNLIMITED);
             prop.minCpu = getDoubleProperty(IGNITE_RESOURCE_MIN_CPU_CNT_PER_NODE, props, DEFAULT_RESOURCE_MIN_CPU);
             prop.minMemory = getDoubleProperty(IGNITE_RESOURCE_MIN_MEMORY_PER_NODE, props, DEFAULT_RESOURCE_MIN_MEM);
@@ -260,4 +336,32 @@ public class ClusterProperties {
 
         return property == null ? defaultVal : property;
     }
+
+    /**
+     * Finds a local, non-loopback, IPv4 address
+     *
+     * @return The first non-loopback IPv4 address found, or
+     *         <code>null</code> if no such addresses found
+     * @throws java.net.SocketException
+     *            If there was a problem querying the network
+     *            interfaces
+     */
+    public static String getNonLoopbackAddress() throws SocketException {
+        Enumeration<NetworkInterface> ifaces = NetworkInterface.getNetworkInterfaces();
+
+        while (ifaces.hasMoreElements()) {
+            NetworkInterface iface = ifaces.nextElement();
+
+            Enumeration<InetAddress> addresses = iface.getInetAddresses();
+
+            while (addresses.hasMoreElements()) {
+                InetAddress addr = addresses.nextElement();
+
+                if (addr instanceof Inet4Address && !addr.isLoopbackAddress())
+                    return addr.getHostAddress();
+            }
+        }
+
+        throw new RuntimeException("Failed. Couldn't find non-loopback address");
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/76a0d0ae/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
index 0ff945b..6cc43b1 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
@@ -27,48 +27,51 @@ import org.glassfish.jersey.server.*;
 import java.net.*;
 
 /**
- * TODO
+ * Ignite mesos framework.
  */
 public class IgniteFramework {
 
-    public static final String IGNITE_FRAMEWORK_NAME = "IgniteFramework";
+    public static final String IGNITE_FRAMEWORK_NAME = "Ignite";
 
     /**
-     * @param args Args
+     * Main methods has only one optional parameter - path to properties files.
+     *
+     * @param args Args.
      */
     public static void main(String[] args) throws Exception {
         final int frameworkFailoverTimeout = 0;
 
+        // Have Mesos fill in the current user.
         Protos.FrameworkInfo.Builder frameworkBuilder = Protos.FrameworkInfo.newBuilder()
             .setName(IGNITE_FRAMEWORK_NAME)
-            .setUser("") // Have Mesos fill in the current user.
-            .setFailoverTimeout(frameworkFailoverTimeout); // timeout in seconds
+            .setUser("")
+            .setFailoverTimeout(frameworkFailoverTimeout);
 
         if (System.getenv("MESOS_CHECKPOINT") != null) {
             System.out.println("Enabling checkpoint for the framework");
             frameworkBuilder.setCheckpoint(true);
         }
 
-        ClusterProperties clusterProperties = ClusterProperties.from(args.length == 1 ? args[0] : null);
+        ClusterProperties clusterProps = ClusterProperties.from(args.length == 1 ? args[0] : null);
 
-        String baseUrl = String.format("http://%s:%d", formatInetAddress(InetAddress.getLocalHost()), 4444);
+        String baseUrl = String.format("http://%s:%d", clusterProps.httpServerHost(), clusterProps.httpServerPort());
 
         URI httpServerBaseUri = URI.create(baseUrl);
 
         ResourceConfig rc = new ResourceConfig()
-            .registerInstances(new ResourceController(clusterProperties.userLibs(), clusterProperties.igniteCfg(),
-                clusterProperties.igniteWorkDir()));
+            .registerInstances(new ResourceController(clusterProps.userLibs(), clusterProps.igniteCfg(),
+                    clusterProps.igniteWorkDir()));
 
         HttpServer httpServer = GrizzlyHttpServerFactory.createHttpServer(httpServerBaseUri, rc);
 
         ResourceProvider provider = new ResourceProvider();
 
-        IgniteProvider igniteProvider = new IgniteProvider(clusterProperties.igniteWorkDir());
+        IgniteProvider igniteProvider = new IgniteProvider(clusterProps.igniteWorkDir());
 
-        provider.init(clusterProperties, igniteProvider, baseUrl);
+        provider.init(clusterProps, igniteProvider, baseUrl);
 
         // Create the scheduler.
-        Scheduler scheduler = new IgniteScheduler(clusterProperties, provider);
+        Scheduler scheduler = new IgniteScheduler(clusterProps, provider);
 
         // create the driver
         MesosSchedulerDriver driver;
@@ -92,13 +95,13 @@ public class IgniteFramework {
 
             frameworkBuilder.setPrincipal(System.getenv("DEFAULT_PRINCIPAL"));
 
-            driver = new MesosSchedulerDriver(scheduler, frameworkBuilder.build(), clusterProperties.masterUrl(),
+            driver = new MesosSchedulerDriver(scheduler, frameworkBuilder.build(), clusterProps.masterUrl(),
                 credential);
         }
         else {
             frameworkBuilder.setPrincipal("ignite-framework-java");
 
-            driver = new MesosSchedulerDriver(scheduler, frameworkBuilder.build(), clusterProperties.masterUrl());
+            driver = new MesosSchedulerDriver(scheduler, frameworkBuilder.build(), clusterProps.masterUrl());
         }
 
         int status = driver.run() == Protos.Status.DRIVER_STOPPED ? 0 : 1;
@@ -110,16 +113,4 @@ public class IgniteFramework {
 
         System.exit(status);
     }
-
-    public static String formatInetAddress(final InetAddress inetAddress) {
-        if (inetAddress instanceof Inet4Address) {
-            return inetAddress.getHostAddress();
-        }
-        else if (inetAddress instanceof Inet6Address) {
-            return String.format("[%s]", inetAddress.getHostAddress());
-        }
-        else
-            throw new IllegalArgumentException("InetAddress type: " + inetAddress.getClass().getName() +
-                " is not supported");
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/76a0d0ae/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index b1ff930..cd517fb 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -25,7 +25,7 @@ import java.util.*;
 import java.util.concurrent.atomic.*;
 
 /**
- * TODO
+ * Ignite scheduler receives offers from Mesos and decides how many resources will be occupied.
  */
 public class IgniteScheduler implements Scheduler {
     /** Cpus. */
@@ -162,7 +162,7 @@ public class IgniteScheduler implements Scheduler {
                 .setName(MEM)
                 .setType(Protos.Value.Type.SCALAR)
                 .setScalar(Protos.Value.Scalar.newBuilder().setValue(igniteTask.mem())))
-            .build();
+                .build();
     }
 
     /**
@@ -187,8 +187,8 @@ public class IgniteScheduler implements Scheduler {
      * @return Ignite task description.
      */
     private IgniteTask checkOffer(Protos.Offer offer) {
-        // Check that limit on running nodes.
-        if (!checkLimit(clusterLimit.instances(), tasks.size()))
+        // Check limit on running nodes.
+        if (clusterLimit.instances() <= tasks.size())
             return null;
 
         double cpus = -1;
@@ -238,12 +238,9 @@ public class IgniteScheduler implements Scheduler {
             totalDisk += task.disk();
         }
 
-        cpus = clusterLimit.cpus() == ClusterProperties.UNLIMITED ? cpus :
-            Math.min(clusterLimit.cpus() - totalCpus, cpus);
-        mem = clusterLimit.memory() == ClusterProperties.UNLIMITED ? mem :
-            Math.min(clusterLimit.memory() - totalMem, mem);
-        disk = clusterLimit.disk() == ClusterProperties.UNLIMITED ? disk :
-            Math.min(clusterLimit.disk() - totalDisk, disk);
+        cpus = Math.min(clusterLimit.cpus() - totalCpus, Math.min(cpus, clusterLimit.cpusPerNode()));
+        mem = Math.min(clusterLimit.memory() - totalMem, Math.min(mem, clusterLimit.memoryPerNode()));
+        disk = Math.min(clusterLimit.disk() - totalDisk, Math.min(disk, clusterLimit.diskPerNode()));
 
         if (cpus > 0 && mem > 0)
             return new IgniteTask(offer.getHostname(), cpus, mem, disk);
@@ -258,15 +255,6 @@ public class IgniteScheduler implements Scheduler {
         }
     }
 
-    /**
-     * @param limit Limit.
-     * @param value Value.
-     * @return {@code True} if limit isn't violated else {@code false}.
-     */
-    private boolean checkLimit(double limit, double value) {
-        return limit == ClusterProperties.UNLIMITED || limit <= value;
-    }
-
     /** {@inheritDoc} */
     @Override public void offerRescinded(SchedulerDriver schedulerDriver, Protos.OfferID offerID) {
         log.info("offerRescinded()");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/76a0d0ae/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteTask.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteTask.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteTask.java
index bad9996..c41ff49 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteTask.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteTask.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.mesos;
 
 /**
- * TODO
+ * Information about launched task.
  */
 public class IgniteTask {
     /** */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/76a0d0ae/modules/mesos/src/main/java/org/apache/ignite/mesos/package-info.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/package-info.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/package-info.java
index 49ddf86..0404c02 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/package-info.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/package-info.java
@@ -17,6 +17,6 @@
 
 /**
  * <!-- Package description. -->
- * Messo Framework.
+ * Contains classes to support integration with Apache Mesos.
  */
 package org.apache.ignite.mesos;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/76a0d0ae/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/IgniteProvider.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/IgniteProvider.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/IgniteProvider.java
index 18ceb00..9a27539 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/IgniteProvider.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/IgniteProvider.java
@@ -23,7 +23,7 @@ import java.nio.channels.*;
 import java.util.*;
 
 /**
- * TODO
+ * Class downloads and stores Ignite.
  */
 public class IgniteProvider {
     /** */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/76a0d0ae/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceController.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceController.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceController.java
index 5c9e693..8f0a2af 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceController.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceController.java
@@ -22,7 +22,7 @@ import javax.ws.rs.core.*;
 import java.io.*;
 
 /**
- *
+ * HTTP controller which provides on slave resources.
  */
 @Path("/")
 public class ResourceController {
@@ -58,9 +58,8 @@ public class ResourceController {
     }
 
     /**
-     *
-     * @param ignite
-     * @return
+     * @param ignite Ignite jar name.
+     * @return Http response.
      */
     @GET
     @Path(IGNITE_PREFIX + "{ignite-dist}")
@@ -69,9 +68,8 @@ public class ResourceController {
     }
 
     /**
-     *
-     * @param lib
-     * @return
+     * @param lib user's jar.
+     * @return Http response.
      */
     @GET
     @Path(LIBS_PREFIX + "{lib}")
@@ -81,8 +79,8 @@ public class ResourceController {
 
     /**
      *
-     * @param cfg
-     * @return
+     * @param cfg Config file.
+     * @return Http response.
      */
     @GET
     @Path(CONFIG_PREFIX + "{cfg}")
@@ -91,9 +89,8 @@ public class ResourceController {
     }
 
     /**
-     *
-     * @param cfg
-     * @return
+     * @param cfg Config file.
+     * @return Http response.
      */
     @GET
     @Path(DEFAULT_CONFIG + "{cfg}")
@@ -111,7 +108,7 @@ public class ResourceController {
      */
     private static Response handleRequest(File resource, String type, String attachmentName) {
         final Response.ResponseBuilder builder = Response.ok(resource, type);
-        builder.header("Content-Disposition", String.format("attachment; filename=\"%s\"", attachmentName));
+        builder.header("Content-Disposition", "attachment; filename=\"" + attachmentName + "\"");
         return builder.build();
     }
 
@@ -124,7 +121,7 @@ public class ResourceController {
      */
     private static Response handleRequest(InputStream resource, String type, String attachmentName) {
         final Response.ResponseBuilder builder = Response.ok(resource, type);
-        builder.header("Content-Disposition", String.format("attachment; filename=\"%s\"", attachmentName));
+        builder.header("Content-Disposition", "attachment; filename=\"" + attachmentName + "\"");
         return builder.build();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/76a0d0ae/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceProvider.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceProvider.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceProvider.java
index 544b15c..1b1f615 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceProvider.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceProvider.java
@@ -25,7 +25,7 @@ import java.util.*;
 import static org.apache.ignite.mesos.resource.ResourceController.*;
 
 /**
- *
+ * Provides path to user's libs and config file.
  */
 public class ResourceProvider {
     /** Ignite url. */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/76a0d0ae/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/package-info.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/package-info.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/package-info.java
new file mode 100644
index 0000000..7e3614e
--- /dev/null
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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 description. -->
+ * Contains classes provide access to resources.
+ */
+package org.apache.ignite.mesos.resource;
\ No newline at end of file


[08/37] incubator-ignite git commit: #IGNITE-857 Added resource limit.

Posted by sb...@apache.org.
#IGNITE-857 Added resource limit.


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

Branch: refs/heads/ignite-gg-10369
Commit: ae8bcf83b5e14efeb21dbd541ef56c629c8e214d
Parents: e320873
Author: nikolay tikhonov <nt...@gridgain.com>
Authored: Wed May 20 11:50:39 2015 +0300
Committer: nikolay tikhonov <nt...@gridgain.com>
Committed: Wed May 20 11:50:39 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/mesos/ClusterResources.java   | 38 ++++++++-
 .../apache/ignite/mesos/IgniteFramework.java    |  4 +-
 .../apache/ignite/mesos/IgniteScheduler.java    | 87 +++++++++++++++-----
 .../ignite/mesos/IgniteSchedulerSelfTest.java   |  2 +-
 4 files changed, 103 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ae8bcf83/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterResources.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterResources.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterResources.java
index 0a2193f..1887530 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterResources.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterResources.java
@@ -52,6 +52,18 @@ public class ClusterResources {
     private double nodeCnt = DEFAULT_VALUE;
 
     /** */
+    public static final String IGNITE_RESOURCE_MIN_CPU_CNT_PER_NODE = "IGNITE_RESOURCE_MIN_CPU_CNT_PER_NODE";
+
+    /** Min memory per node. */
+    private int minCpu = 2;
+
+    /** */
+    public static final String IGNITE_RESOURCE_MIN_MEMORY_PER_NODE = "IGNITE_RESOURCE_MIN_MEMORY_PER_NODE";
+
+    /** Min memory per node. */
+    private int minMemoryCnt = 256;
+
+    /** */
     public ClusterResources() {
         // No-op.
     }
@@ -85,14 +97,32 @@ public class ClusterResources {
     }
 
     /**
+     * @return min memory per node.
+     */
+    public int minMemoryPerNode() {
+        return minMemoryCnt;
+    }
+
+    /**
+     * @return min cpu count per node.
+     */
+    public int minCpuPerNode() {
+        return minCpu;
+    }
+
+    /**
      * @param config path to config file.
      * @return Cluster configuration.
      */
     public static ClusterResources from(String config) {
         try {
-            Properties props = new Properties();
+            Properties props = null;
+
+            if (config != null) {
+                props = new Properties();
 
-            props.load(new FileInputStream(config));
+                props.load(new FileInputStream(config));
+            }
 
             ClusterResources resources = new ClusterResources();
 
@@ -114,13 +144,13 @@ public class ClusterResources {
      * @return Property value.
      */
     private static double getProperty(String name, Properties fileProps) {
-        if (fileProps.containsKey(name))
+        if (fileProps != null && fileProps.containsKey(name))
             return Double.valueOf(fileProps.getProperty(name));
 
         String property = System.getProperty(name);
 
         if (property == null)
-            System.getenv(name);
+            property = System.getenv(name);
 
         if (property == null)
             return DEFAULT_VALUE;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ae8bcf83/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
index 3d309f3..2d74f71 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
@@ -42,8 +42,8 @@ public class IgniteFramework {
             frameworkBuilder.setCheckpoint(true);
         }
 
-        // create the scheduler
-        final Scheduler scheduler = new IgniteScheduler(ClusterResources.from(args[1]));
+        // Create the scheduler.
+        final Scheduler scheduler = new IgniteScheduler(ClusterResources.from(null));
 
         // create the driver
         MesosSchedulerDriver driver;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ae8bcf83/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index fcbab87..9d10860 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -21,7 +21,6 @@ import org.apache.mesos.*;
 import org.slf4j.*;
 
 import java.util.*;
-import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
 /**
@@ -46,9 +45,6 @@ public class IgniteScheduler implements Scheduler {
     /** Default port range. */
     public static final String DEFAULT_PORT = ":47500..47510";
 
-    /** Min of memory required. */
-    public static final int MIN_MEMORY = 256;
-
     /** Delimiter to use in IP names. */
     public static final String DELIM = ",";
 
@@ -62,7 +58,7 @@ public class IgniteScheduler implements Scheduler {
     private AtomicInteger taskIdGenerator = new AtomicInteger();
 
     /** Task on host. */
-    private ConcurrentMap<String, IgniteTask> tasks = new ConcurrentHashMap<>();
+    private Map<String, IgniteTask> tasks = new HashMap<>();
 
     /** Cluster resources. */
     private ClusterResources clusterLimit;
@@ -82,7 +78,7 @@ public class IgniteScheduler implements Scheduler {
 
     /** {@inheritDoc} */
     @Override public void reregistered(SchedulerDriver schedulerDriver, Protos.MasterInfo masterInfo) {
-        log.info("reregistered");
+        log.info("reregistered()");
     }
 
     /** {@inheritDoc} */
@@ -138,7 +134,7 @@ public class IgniteScheduler implements Scheduler {
         cont.setDocker(docker.build());
 
         return Protos.TaskInfo.newBuilder()
-            .setName("task " + taskId.getValue())
+            .setName("Ignite node " + taskId.getValue())
             .setTaskId(taskId)
             .setSlaveId(offer.getSlaveId())
             .addResources(Protos.Resource.newBuilder()
@@ -153,7 +149,7 @@ public class IgniteScheduler implements Scheduler {
             .setCommand(Protos.CommandInfo.newBuilder()
                 .setShell(false)
                 .addArguments(STARTUP_SCRIPT)
-                .addArguments(String.valueOf(igniteTask.mem()))
+                .addArguments(String.valueOf((int) igniteTask.mem()))
                 .addArguments(getAddress()))
             .build();
     }
@@ -180,13 +176,15 @@ public class IgniteScheduler implements Scheduler {
      * @return Ignite task description.
      */
     private IgniteTask checkOffer(Protos.Offer offer) {
-        if (checkLimit(clusterLimit.instances(), tasks.size()))
+        // Check that limit on running nodes.
+        if (!checkLimit(clusterLimit.instances(), tasks.size()))
             return null;
 
-        double cpus = -2;
-        double mem = -2;
-        double disk = -2;
+        double cpus = -1;
+        double mem = -1;
+        double disk = -1;
 
+        // Collect resource on slave.
         for (Protos.Resource resource : offer.getResourcesList()) {
             if (resource.getName().equals(CPUS)) {
                 if (resource.getType().equals(Protos.Value.Type.SCALAR))
@@ -200,17 +198,43 @@ public class IgniteScheduler implements Scheduler {
                 else
                     log.debug("Mem resource was not a scalar: " + resource.getType().toString());
             }
-            else if (resource.getType().equals(Protos.Value.Type.SCALAR))
-                disk = resource.getScalar().getValue();
-            else
-                log.debug("Disk resource was not a scalar: " + resource.getType().toString());
+            else if (resource.getName().equals(DISK))
+                if (resource.getType().equals(Protos.Value.Type.SCALAR))
+                    disk = resource.getScalar().getValue();
+                else
+                    log.debug("Disk resource was not a scalar: " + resource.getType().toString());
+        }
+
+        // Check that slave satisfies min requirements.
+        if (cpus < clusterLimit.minCpuPerNode()  && mem < clusterLimit.minMemoryPerNode() ) {
+            log.info("Offer not sufficient for slave request:\n" + offer.getResourcesList().toString() +
+                "\n" + offer.getAttributesList().toString() +
+                "\nRequested for slave:\n" +
+                "  cpus:  " + cpus + "\n" +
+                "  mem:   " + mem);
+
+            return null;
         }
 
-        if (checkLimit(clusterLimit.memory(), mem) &&
-            checkLimit(clusterLimit.cpus(), cpus) &&
-            checkLimit(clusterLimit.disk(), disk) &&
-            MIN_MEMORY <= mem)
+        double totalCpus = 0;
+        double totalMem = 0;
+        double totalDisk = 0;
 
+        // Collect occupied resources.
+        for (IgniteTask task : tasks.values()) {
+            totalCpus += task.cpuCores();
+            totalMem += task.mem();
+            totalDisk += task.disk();
+        }
+
+        cpus = clusterLimit.cpus() == ClusterResources.DEFAULT_VALUE ? cpus :
+            Math.min(clusterLimit.cpus() - totalCpus, cpus);
+        mem = clusterLimit.memory() == ClusterResources.DEFAULT_VALUE ? mem :
+            Math.min(clusterLimit.memory() - totalMem, mem);
+        disk = clusterLimit.disk() == ClusterResources.DEFAULT_VALUE ? disk :
+            Math.min(clusterLimit.disk() - totalDisk, disk);
+
+        if (cpus > 0 && mem > 0)
             return new IgniteTask(offer.getHostname(), cpus, mem, disk);
         else {
             log.info("Offer not sufficient for slave request:\n" + offer.getResourcesList().toString() +
@@ -246,7 +270,28 @@ public class IgniteScheduler implements Scheduler {
         switch (taskStatus.getState()) {
             case TASK_FAILED:
             case TASK_FINISHED:
-                tasks.remove(taskId);
+                synchronized (mux) {
+                    IgniteTask failedTask = tasks.remove(taskId);
+
+                    if (failedTask != null) {
+                        List<Protos.Request> requests = new ArrayList<>();
+
+                        Protos.Request request = Protos.Request.newBuilder()
+                            .addResources(Protos.Resource.newBuilder()
+                                .setType(Protos.Value.Type.SCALAR)
+                                .setName(MEM)
+                                .setScalar(Protos.Value.Scalar.newBuilder().setValue(failedTask.mem())))
+                            .addResources(Protos.Resource.newBuilder()
+                                .setType(Protos.Value.Type.SCALAR)
+                                .setName(CPUS)
+                                .setScalar(Protos.Value.Scalar.newBuilder().setValue(failedTask.cpuCores())))
+                            .build();
+
+                        requests.add(request);
+
+                        schedulerDriver.requestResources(requests);
+                    }
+                }
                 break;
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ae8bcf83/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java b/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
index 2c4b6ee..8f8ca8b 100644
--- a/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
+++ b/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
@@ -33,7 +33,7 @@ public class IgniteSchedulerSelfTest extends TestCase {
     @Override public void setUp() throws Exception {
         super.setUp();
 
-        scheduler = new IgniteScheduler();
+        //scheduler = new IgniteScheduler();
     }
 
     /**


[14/37] incubator-ignite git commit: #IGNITE-857 Removed Grizzly dependecy. Added tests.

Posted by sb...@apache.org.
#IGNITE-857 Removed Grizzly dependecy. Added tests.


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

Branch: refs/heads/ignite-gg-10369
Commit: 639c912870dccde404d99407c445dfe723b2bdba
Parents: 5e3bcb2
Author: nikolay tikhonov <nt...@gridgain.com>
Authored: Tue May 26 16:05:25 2015 +0300
Committer: nikolay tikhonov <nt...@gridgain.com>
Committed: Tue May 26 16:05:25 2015 +0300

----------------------------------------------------------------------
 modules/mesos/licenses/jetty-epl-license.txt    |  69 ++++++
 modules/mesos/pom.xml                           |  19 +-
 .../apache/ignite/mesos/ClusterProperties.java  |  37 ++-
 .../apache/ignite/mesos/IgniteFramework.java    |  16 +-
 .../apache/ignite/mesos/IgniteScheduler.java    | 107 ++++----
 .../org/apache/ignite/mesos/IgniteTask.java     |   8 +
 .../ignite/mesos/resource/IgniteProvider.java   |   6 +-
 .../ignite/mesos/resource/JettyServer.java      |  61 +++++
 .../mesos/resource/ResourceController.java      | 127 ----------
 .../ignite/mesos/resource/ResourceHandler.java  | 142 +++++++++++
 .../ignite/mesos/resource/ResourceProvider.java |   2 +-
 .../main/resources/ignite-default-config.xml    |   6 +-
 .../ignite/mesos/IgniteSchedulerSelfTest.java   | 242 ++++++++++++++++++-
 13 files changed, 636 insertions(+), 206 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/639c9128/modules/mesos/licenses/jetty-epl-license.txt
----------------------------------------------------------------------
diff --git a/modules/mesos/licenses/jetty-epl-license.txt b/modules/mesos/licenses/jetty-epl-license.txt
new file mode 100644
index 0000000..f5f0c89
--- /dev/null
+++ b/modules/mesos/licenses/jetty-epl-license.txt
@@ -0,0 +1,69 @@
+Eclipse Public License, Version 1.0 (EPL-1.0)
+(plain text)
+THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
+
+1. DEFINITIONS
+
+"Contribution" means:
+
+a) in the case of the initial Contributor, the initial code and documentation distributed under this Agreement, and
+b) in the case of each subsequent Contributor:
+i) changes to the Program, and
+ii) additions to the Program;
+where such changes and/or additions to the Program originate from and are distributed by that particular Contributor. A Contribution 'originates' from a Contributor if it was added to the Program by such Contributor itself or anyone acting on such Contributor's behalf. Contributions do not include additions to the Program which: (i) are separate modules of software distributed in conjunction with the Program under their own license agreement, and (ii) are not derivative works of the Program.
+"Contributor" means any person or entity that distributes the Program.
+
+"Licensed Patents " mean patent claims licensable by a Contributor which are necessarily infringed by the use or sale of its Contribution alone or when combined with the Program.
+
+"Program" means the Contributions distributed in accordance with this Agreement.
+
+"Recipient" means anyone who receives the Program under this Agreement, including all Contributors.
+
+2. GRANT OF RIGHTS
+
+a) Subject to the terms of this Agreement, each Contributor hereby grants Recipient a non-exclusive, worldwide, royalty-free copyright license to reproduce, prepare derivative works of, publicly display, publicly perform, distribute and sublicense the Contribution of such Contributor, if any, and such derivative works, in source code and object code form.
+b) Subject to the terms of this Agreement, each Contributor hereby grants Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed Patents to make, use, sell, offer to sell, import and otherwise transfer the Contribution of such Contributor, if any, in source code and object code form. This patent license shall apply to the combination of the Contribution and the Program if, at the time the Contribution is added by the Contributor, such addition of the Contribution causes such combination to be covered by the Licensed Patents. The patent license shall not apply to any other combinations which include the Contribution. No hardware per se is licensed hereunder.
+c) Recipient understands that although each Contributor grants the licenses to its Contributions set forth herein, no assurances are provided by any Contributor that the Program does not infringe the patent or other intellectual property rights of any other entity. Each Contributor disclaims any liability to Recipient for claims brought by any other entity based on infringement of intellectual property rights or otherwise. As a condition to exercising the rights and licenses granted hereunder, each Recipient hereby assumes sole responsibility to secure any other intellectual property rights needed, if any. For example, if a third party patent license is required to allow Recipient to distribute the Program, it is Recipient's responsibility to acquire that license before distributing the Program.
+d) Each Contributor represents that to its knowledge it has sufficient copyright rights in its Contribution, if any, to grant the copyright license set forth in this Agreement.
+3. REQUIREMENTS
+
+A Contributor may choose to distribute the Program in object code form under its own license agreement, provided that:
+
+a) it complies with the terms and conditions of this Agreement; and
+b) its license agreement:
+i) effectively disclaims on behalf of all Contributors all warranties and conditions, express and implied, including warranties or conditions of title and non-infringement, and implied warranties or conditions of merchantability and fitness for a particular purpose;
+ii) effectively excludes on behalf of all Contributors all liability for damages, including direct, indirect, special, incidental and consequential damages, such as lost profits;
+iii) states that any provisions which differ from this Agreement are offered by that Contributor alone and not by any other party; and
+iv) states that source code for the Program is available from such Contributor, and informs licensees how to obtain it in a reasonable manner on or through a medium customarily used for software exchange.
+When the Program is made available in source code form:
+
+a) it must be made available under this Agreement; and
+b) a copy of this Agreement must be included with each copy of the Program.
+Contributors may not remove or alter any copyright notices contained within the Program.
+Each Contributor must identify itself as the originator of its Contribution, if any, in a manner that reasonably allows subsequent Recipients to identify the originator of the Contribution.
+
+4. COMMERCIAL DISTRIBUTION
+
+Commercial distributors of software may accept certain responsibilities with respect to end users, business partners and the like. While this license is intended to facilitate the commercial use of the Program, the Contributor who includes the Program in a commercial product offering should do so in a manner which does not create potential liability for other Contributors. Therefore, if a Contributor includes the Program in a commercial product offering, such Contributor ("Commercial Contributor") hereby agrees to defend and indemnify every other Contributor ("Indemnified Contributor") against any losses, damages and costs (collectively "Losses") arising from claims, lawsuits and other legal actions brought by a third party against the Indemnified Contributor to the extent caused by the acts or omissions of such Commercial Contributor in connection with its distribution of the Program in a commercial product offering. The obligations in this section do not apply to any claims or Los
 ses relating to any actual or alleged intellectual property infringement. In order to qualify, an Indemnified Contributor must: a) promptly notify the Commercial Contributor in writing of such claim, and b) allow the Commercial Contributor to control, and cooperate with the Commercial Contributor in, the defense and any related settlement negotiations. The Indemnified Contributor may participate in any such claim at its own expense.
+
+For example, a Contributor might include the Program in a commercial product offering, Product X. That Contributor is then a Commercial Contributor. If that Commercial Contributor then makes performance claims, or offers warranties related to Product X, those performance claims and warranties are such Commercial Contributor's responsibility alone. Under this section, the Commercial Contributor would have to defend claims against the other Contributors related to those performance claims and warranties, and if a court requires any other Contributor to pay any damages as a result, the Commercial Contributor must pay those damages.
+
+5. NO WARRANTY
+
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each Recipient is solely responsible for determining the appropriateness of using and distributing the Program and assumes all risks associated with its exercise of rights under this Agreement , including but not limited to the risks and costs of program errors, compliance with applicable laws, damage to or loss of data, programs or equipment, and unavailability or interruption of operations.
+
+6. DISCLAIMER OF LIABILITY
+
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+7. GENERAL
+
+If any provision of this Agreement is invalid or unenforceable under applicable law, it shall not affect the validity or enforceability of the remainder of the terms of this Agreement, and without further action by the parties hereto, such provision shall be reformed to the minimum extent necessary to make such provision valid and enforceable.
+
+If Recipient institutes patent litigation against any entity (including a cross-claim or counterclaim in a lawsuit) alleging that the Program itself (excluding combinations of the Program with other software or hardware) infringes such Recipient's patent(s), then such Recipient's rights granted under Section 2(b) shall terminate as of the date such litigation is filed.
+
+All Recipient's rights under this Agreement shall terminate if it fails to comply with any of the material terms or conditions of this Agreement and does not cure such failure in a reasonable period of time after becoming aware of such noncompliance. If all Recipient's rights under this Agreement terminate, Recipient agrees to cease use and distribution of the Program as soon as reasonably practicable. However, Recipient's obligations under this Agreement and any licenses granted by Recipient relating to the Program shall continue and survive.
+
+Everyone is permitted to copy and distribute copies of this Agreement, but in order to avoid inconsistency the Agreement is copyrighted and may only be modified in the following manner. The Agreement Steward reserves the right to publish new versions (including revisions) of this Agreement from time to time. No one other than the Agreement Steward has the right to modify this Agreement. The Eclipse Foundation is the initial Agreement Steward. The Eclipse Foundation may assign the responsibility to serve as the Agreement Steward to a suitable separate entity. Each new version of the Agreement will be given a distinguishing version number. The Program (including Contributions) may always be distributed subject to the version of the Agreement under which it was received. In addition, after a new version of the Agreement is published, Contributor may elect to distribute the Program (including its Contributions) under the new version. Except as expressly stated in Sections 2(a) and 2(b) 
 above, Recipient receives no rights or licenses to the intellectual property of any Contributor under this Agreement, whether expressly, by implication, estoppel or otherwise. All rights in the Program not expressly granted under this Agreement are reserved.
+
+This Agreement is governed by the laws of the State of New York and the intellectual property laws of the United States of America. No party to this Agreement will bring a legal action under this Agreement more than one year after the cause of action arose. Each party waives its rights to a jury trial in any resulting litigation.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/639c9128/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index 4aa0dae..9079c66 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -28,38 +28,41 @@
     <version>1.1.0-SNAPSHOT</version>
 
     <properties>
-        <version.grizzly>2.16</version.grizzly>
+        <jetty.version>9.2.10.v20150310</jetty.version>
+        <mesos.version>0.22.0</mesos.version>
+        <slf4j.version>1.7.12</slf4j.version>
+        <log4j.version>2.0.2</log4j.version>
     </properties>
 
     <dependencies>
         <dependency>
             <groupId>org.apache.mesos</groupId>
             <artifactId>mesos</artifactId>
-            <version>0.22.0</version>
+            <version>${mesos.version}</version>
         </dependency>
 
         <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-api</artifactId>
-            <version>1.7.12</version>
+            <version>${slf4j.version}</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.logging.log4j</groupId>
             <artifactId>log4j-core</artifactId>
-            <version>2.0.2</version>
+            <version>${log4j.version}</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.logging.log4j</groupId>
             <artifactId>log4j-slf4j-impl</artifactId>
-            <version>2.0.2</version>
+            <version>${log4j.version}</version>
         </dependency>
 
         <dependency>
-            <groupId>org.glassfish.jersey.containers</groupId>
-            <artifactId>jersey-container-grizzly2-http</artifactId>
-            <version>${version.grizzly}</version>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-server</artifactId>
+            <version>${jetty.version}</version>
         </dependency>
 
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/639c9128/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
index 63ef27c..de0afcf 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
@@ -98,7 +98,7 @@ public class ClusterProperties {
     public static final String IGNITE_RESOURCE_MIN_CPU_CNT_PER_NODE = "IGNITE_RESOURCE_MIN_CPU_CNT_PER_NODE";
 
     /** */
-    public static final double DEFAULT_RESOURCE_MIN_CPU = 2;
+    public static final double DEFAULT_RESOURCE_MIN_CPU = 1;
 
     /** Min memory per node. */
     private double minCpu = DEFAULT_RESOURCE_MIN_CPU;
@@ -155,6 +155,13 @@ public class ClusterProperties {
     }
 
     /**
+     * Set CPU count limit.
+     */
+    public void cpus(double cpu){
+        this.cpu = cpu;
+    }
+
+    /**
      * @return CPU count limit.
      */
     public double cpusPerNode(){
@@ -169,6 +176,14 @@ public class ClusterProperties {
     }
 
     /**
+     * Set mem limit.
+     */
+    public void memory(double mem) {
+        this.mem = mem;
+    }
+
+
+    /**
      * @return mem limit.
      */
     public double memoryPerNode() {
@@ -204,6 +219,15 @@ public class ClusterProperties {
     }
 
     /**
+     * Sets min memory.
+     *
+     * @param minMemory Min memory.
+     */
+    public void minMemoryPerNode(double minMemory) {
+        this.minMemory = minMemory;
+    }
+
+    /**
      * @return min cpu count per node.
      */
     public double minCpuPerNode() {
@@ -211,6 +235,15 @@ public class ClusterProperties {
     }
 
     /**
+     * Sets min cpu count per node.
+     *
+     * @param minCpu min cpu count per node.
+     */
+    public void minCpuPerNode(double minCpu) {
+        this.minCpu = minCpu;
+    }
+
+    /**
      * @return Ignite version.
      */
     public String igniteVer() {
@@ -286,7 +319,7 @@ public class ClusterProperties {
             prop.mem = getDoubleProperty(IGNITE_RESOURCE_MEM_MB, props, UNLIMITED);
             prop.memPerNode = getDoubleProperty(IGNITE_RESOURCE_MEM_MB_PER_NODE, props, UNLIMITED);
             prop.disk = getDoubleProperty(IGNITE_RESOURCE_DISK_MB, props, UNLIMITED);
-            prop.diskPerNode = getDoubleProperty(IGNITE_RESOURCE_DISK_MB_PER_NODE, props, UNLIMITED);
+            prop.diskPerNode = getDoubleProperty(IGNITE_RESOURCE_DISK_MB_PER_NODE, props, 1024.0);
             prop.nodeCnt = getDoubleProperty(IGNITE_RESOURCE_NODE_CNT, props, UNLIMITED);
             prop.minCpu = getDoubleProperty(IGNITE_RESOURCE_MIN_CPU_CNT_PER_NODE, props, DEFAULT_RESOURCE_MIN_CPU);
             prop.minMemory = getDoubleProperty(IGNITE_RESOURCE_MIN_MEMORY_PER_NODE, props, DEFAULT_RESOURCE_MIN_MEM);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/639c9128/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
index b385bc9..154385b 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
@@ -20,9 +20,6 @@ package org.apache.ignite.mesos;
 import com.google.protobuf.*;
 import org.apache.ignite.mesos.resource.*;
 import org.apache.mesos.*;
-import org.glassfish.grizzly.http.server.*;
-import org.glassfish.jersey.grizzly2.httpserver.*;
-import org.glassfish.jersey.server.*;
 import org.slf4j.*;
 
 import java.net.*;
@@ -61,13 +58,12 @@ public class IgniteFramework {
 
         String baseUrl = String.format("http://%s:%d", clusterProps.httpServerHost(), clusterProps.httpServerPort());
 
-        URI httpServerBaseUri = URI.create(baseUrl);
+        JettyServer httpServer = new JettyServer();
 
-        ResourceConfig rc = new ResourceConfig()
-            .registerInstances(new ResourceController(clusterProps.userLibs(), clusterProps.igniteCfg(),
-                clusterProps.igniteWorkDir()));
-
-        HttpServer httpServer = GrizzlyHttpServerFactory.createHttpServer(httpServerBaseUri, rc);
+        httpServer.start(
+            new InetSocketAddress(clusterProps.httpServerHost(), clusterProps.httpServerPort()),
+            new ResourceHandler(clusterProps.userLibs(), clusterProps.igniteCfg(), clusterProps.igniteWorkDir())
+        );
 
         ResourceProvider provider = new ResourceProvider();
 
@@ -113,7 +109,7 @@ public class IgniteFramework {
 
         int status = driver.run() == Protos.Status.DRIVER_STOPPED ? 0 : 1;
 
-        httpServer.shutdown();
+        httpServer.stop();
 
         // Ensure that the driver process terminates.
         driver.stop();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/639c9128/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index 7d713cd..6b165e4 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -56,24 +56,24 @@ public class IgniteScheduler implements Scheduler {
     private Map<String, IgniteTask> tasks = new HashMap<>();
 
     /** Cluster resources. */
-    private ClusterProperties clusterLimit;
+    private ClusterProperties clusterProps;
 
     /** Resource provider. */
     private ResourceProvider resourceProvider;
 
     /**
-     * @param clusterLimit Cluster limit.
+     * @param clusterProps Cluster limit.
      * @param resourceProvider Resource provider.
      */
-    public IgniteScheduler(ClusterProperties clusterLimit, ResourceProvider resourceProvider) {
-        this.clusterLimit = clusterLimit;
+    public IgniteScheduler(ClusterProperties clusterProps, ResourceProvider resourceProvider) {
+        this.clusterProps = clusterProps;
         this.resourceProvider = resourceProvider;
     }
 
     /** {@inheritDoc} */
     @Override public void resourceOffers(SchedulerDriver schedulerDriver, List<Protos.Offer> offers) {
         synchronized (mux) {
-            log.info("resourceOffers() with {} offers", offers.size());
+            log.debug("Offers resources: {} ", offers.size());
 
             for (Protos.Offer offer : offers) {
                 IgniteTask igniteTask = checkOffer(offer);
@@ -89,7 +89,7 @@ public class IgniteScheduler implements Scheduler {
                 Protos.TaskID taskId = Protos.TaskID.newBuilder()
                     .setValue(Integer.toString(taskIdGenerator.incrementAndGet())).build();
 
-                log.info("Launching task {}", taskId.getValue());
+                log.info("Launching task: [{}]", igniteTask);
 
                 // Create task to run.
                 Protos.TaskInfo task = createTask(offer, igniteTask, taskId);
@@ -103,7 +103,6 @@ public class IgniteScheduler implements Scheduler {
         }
     }
 
-
     /**
      * Create Task.
      *
@@ -116,7 +115,7 @@ public class IgniteScheduler implements Scheduler {
         Protos.CommandInfo.Builder builder = Protos.CommandInfo.newBuilder()
             .setEnvironment(Protos.Environment.newBuilder().addVariables(Protos.Environment.Variable.newBuilder()
                 .setName("IGNITE_TCP_DISCOVERY_ADDRESSES")
-                .setValue(getAddress())))
+                .setValue(getAddress(offer.getHostname()))))
             .addUris(Protos.CommandInfo.URI.newBuilder()
                 .setValue(resourceProvider.igniteUrl())
                 .setExtract(true))
@@ -129,14 +128,14 @@ public class IgniteScheduler implements Scheduler {
             builder.setValue("cp *.jar ./gridgain-community-*/libs/ "
                 + "&& ./gridgain-community-*/bin/ignite.sh "
                 + resourceProvider.configName()
-                + " -J-Xmx" + String.valueOf((int) igniteTask.mem() + "m")
-                + " -J-Xms" + String.valueOf((int) igniteTask.mem()) + "m");
+                + " -J-Xmx" + String.valueOf((int)igniteTask.mem() + "m")
+                + " -J-Xms" + String.valueOf((int)igniteTask.mem()) + "m");
         }
         else
             builder.setValue("./gridgain-community-*/bin/ignite.sh "
                 + resourceProvider.configName()
-                + " -J-Xmx" + String.valueOf((int) igniteTask.mem() + "m")
-                + " -J-Xms" + String.valueOf((int) igniteTask.mem()) + "m");
+                + " -J-Xmx" + String.valueOf((int)igniteTask.mem() + "m")
+                + " -J-Xms" + String.valueOf((int)igniteTask.mem()) + "m");
 
         return Protos.TaskInfo.newBuilder()
             .setName("Ignite node " + taskId.getValue())
@@ -151,15 +150,23 @@ public class IgniteScheduler implements Scheduler {
                 .setName(MEM)
                 .setType(Protos.Value.Type.SCALAR)
                 .setScalar(Protos.Value.Scalar.newBuilder().setValue(igniteTask.mem())))
+            .addResources(Protos.Resource.newBuilder()
+                .setName(DISK)
+                .setType(Protos.Value.Type.SCALAR)
+                .setScalar(Protos.Value.Scalar.newBuilder().setValue(igniteTask.disk())))
                 .build();
     }
 
     /**
      * @return Address running nodes.
      */
-    protected String getAddress() {
-        if (tasks.isEmpty())
+    protected String getAddress(String address) {
+        if (tasks.isEmpty()) {
+            if (address != null && !address.isEmpty())
+                return address + DEFAULT_PORT;
+
             return "";
+        }
 
         StringBuilder sb = new StringBuilder();
 
@@ -177,7 +184,7 @@ public class IgniteScheduler implements Scheduler {
      */
     private IgniteTask checkOffer(Protos.Offer offer) {
         // Check limit on running nodes.
-        if (clusterLimit.instances() <= tasks.size())
+        if (clusterProps.instances() <= tasks.size())
             return null;
 
         double cpus = -1;
@@ -206,7 +213,7 @@ public class IgniteScheduler implements Scheduler {
         }
 
         // Check that slave satisfies min requirements.
-        if (cpus < clusterLimit.minCpuPerNode()  && mem < clusterLimit.minMemoryPerNode() ) {
+        if (cpus < clusterProps.minCpuPerNode() || mem < clusterProps.minMemoryPerNode() ) {
             log.debug("Offer not sufficient for slave request: {}", offer.getResourcesList());
 
             return null;
@@ -223,9 +230,9 @@ public class IgniteScheduler implements Scheduler {
             totalDisk += task.disk();
         }
 
-        cpus = Math.min(clusterLimit.cpus() - totalCpus, Math.min(cpus, clusterLimit.cpusPerNode()));
-        mem = Math.min(clusterLimit.memory() - totalMem, Math.min(mem, clusterLimit.memoryPerNode()));
-        disk = Math.min(clusterLimit.disk() - totalDisk, Math.min(disk, clusterLimit.diskPerNode()));
+        cpus = Math.min(clusterProps.cpus() - totalCpus, Math.min(cpus, clusterProps.cpusPerNode()));
+        mem = Math.min(clusterProps.memory() - totalMem, Math.min(mem, clusterProps.memoryPerNode()));
+        disk = Math.min(clusterProps.disk() - totalDisk, Math.min(disk, clusterProps.diskPerNode()));
 
         if (cpus > 0 && mem > 0)
             return new IgniteTask(offer.getHostname(), cpus, mem, disk);
@@ -240,37 +247,45 @@ public class IgniteScheduler implements Scheduler {
     @Override public void statusUpdate(SchedulerDriver schedulerDriver, Protos.TaskStatus taskStatus) {
         final String taskId = taskStatus.getTaskId().getValue();
 
-        log.info("statusUpdate() task {} is in state {}", taskId, taskStatus.getState());
-
-        switch (taskStatus.getState()) {
-            case TASK_FAILED:
-            case TASK_FINISHED:
-                synchronized (mux) {
-                    IgniteTask failedTask = tasks.remove(taskId);
-
-                    if (failedTask != null) {
-                        List<Protos.Request> requests = new ArrayList<>();
-
-                        Protos.Request request = Protos.Request.newBuilder()
-                            .addResources(Protos.Resource.newBuilder()
-                                .setType(Protos.Value.Type.SCALAR)
-                                .setName(MEM)
-                                .setScalar(Protos.Value.Scalar.newBuilder().setValue(failedTask.mem())))
-                            .addResources(Protos.Resource.newBuilder()
-                                .setType(Protos.Value.Type.SCALAR)
-                                .setName(CPUS)
-                                .setScalar(Protos.Value.Scalar.newBuilder().setValue(failedTask.cpuCores())))
-                            .build();
-
-                        requests.add(request);
-
-                        schedulerDriver.requestResources(requests);
-                    }
+        log.info("Received update event task: [{}] is in state: [{}]", taskId, taskStatus.getState());
+
+        if (taskStatus.getState().equals(Protos.TaskState.TASK_FAILED)
+            || taskStatus.getState().equals(Protos.TaskState.TASK_ERROR)
+            || taskStatus.getState().equals(Protos.TaskState.TASK_FINISHED)
+            || taskStatus.getState().equals(Protos.TaskState.TASK_KILLED)
+            || taskStatus.getState().equals(Protos.TaskState.TASK_LOST)) {
+            synchronized (mux) {
+                IgniteTask failedTask = tasks.remove(taskId);
+
+                if (failedTask != null) {
+                    List<Protos.Request> requests = new ArrayList<>();
+
+                    Protos.Request request = Protos.Request.newBuilder()
+                        .addResources(Protos.Resource.newBuilder()
+                            .setType(Protos.Value.Type.SCALAR)
+                            .setName(MEM)
+                            .setScalar(Protos.Value.Scalar.newBuilder().setValue(failedTask.mem())))
+                        .addResources(Protos.Resource.newBuilder()
+                            .setType(Protos.Value.Type.SCALAR)
+                            .setName(CPUS)
+                            .setScalar(Protos.Value.Scalar.newBuilder().setValue(failedTask.cpuCores())))
+                        .build();
+
+                    requests.add(request);
+
+                    schedulerDriver.requestResources(requests);
                 }
-                break;
+            }
         }
     }
 
+    /**
+     * @param clusterProps Cluster properties.
+     */
+    public void setClusterProps(ClusterProperties clusterProps) {
+        this.clusterProps = clusterProps;
+    }
+
     /** {@inheritDoc} */
     @Override public void registered(SchedulerDriver schedulerDriver, Protos.FrameworkID frameworkID,
         Protos.MasterInfo masterInfo) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/639c9128/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteTask.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteTask.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteTask.java
index c41ff49..ecd2272 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteTask.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteTask.java
@@ -75,4 +75,12 @@ public class IgniteTask {
     public double disk() {
         return disk;
     }
+
+    @Override
+    public String toString() {
+        return "IgniteTask " +
+            "host: [" + host + ']' +
+            ", cpuCores: [" + cpuCores + "]" +
+            ", mem: [" + mem + "]";
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/639c9128/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/IgniteProvider.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/IgniteProvider.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/IgniteProvider.java
index 9a27539..2887112 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/IgniteProvider.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/IgniteProvider.java
@@ -66,9 +66,9 @@ public class IgniteProvider {
                             String[] ver1 = parseVersion(f1).split("\\.");
                             String[] ver2 = parseVersion(f2).split("\\.");
 
-                            if (Integer.valueOf(ver1[0]) > Integer.valueOf(ver2[0])
-                                && Integer.valueOf(ver1[1]) > Integer.valueOf(ver2[1])
-                                && Integer.valueOf(ver1[2]) > Integer.valueOf(ver2[2]))
+                            if (Integer.valueOf(ver1[0]) >= Integer.valueOf(ver2[0])
+                                && Integer.valueOf(ver1[1]) >= Integer.valueOf(ver2[1])
+                                && Integer.valueOf(ver1[2]) >= Integer.valueOf(ver2[2]))
 
                                 return 1;
                             else

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/639c9128/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/JettyServer.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/JettyServer.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/JettyServer.java
new file mode 100644
index 0000000..fb27963
--- /dev/null
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/JettyServer.java
@@ -0,0 +1,61 @@
+/*
+ * 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.mesos.resource;
+
+import org.eclipse.jetty.server.*;
+
+import java.net.*;
+
+/**
+ * Embedded jetty server.
+ */
+public class JettyServer {
+    /** */
+    private Server server;
+
+    /**
+     * Starts jetty server.
+     *
+     * @param address Inter socket address.
+     * @param handler Handler.
+     * @throws Exception If failed.
+     */
+    public void start(InetSocketAddress address, Handler handler) throws Exception {
+        if (server == null) {
+            server = new Server(address);
+
+            server.setHandler(handler);
+
+            server.start();
+        }
+        else
+            throw new IllegalStateException("Failed. Jetty server has been started already.");
+    }
+
+    /**
+     * Stops server.
+     *
+     * @throws Exception If failed.
+     */
+    public void stop() throws Exception {
+        if (server != null)
+            server.stop();
+        else
+            throw new IllegalStateException("Failed. Jetty server has not been started yet.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/639c9128/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceController.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceController.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceController.java
deleted file mode 100644
index 8f0a2af..0000000
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceController.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * 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.mesos.resource;
-
-import javax.ws.rs.*;
-import javax.ws.rs.core.*;
-import java.io.*;
-
-/**
- * HTTP controller which provides on slave resources.
- */
-@Path("/")
-public class ResourceController {
-    /** */
-    public static final String IGNITE_PREFIX = "/ignite/";
-
-    /** */
-    public static final String LIBS_PREFIX = "/libs/";
-
-    /** */
-    public static final String CONFIG_PREFIX = "/config/";
-
-    /** */
-    public static final String DEFAULT_CONFIG = CONFIG_PREFIX + "default/";
-
-    /** */
-    private String libsDir;
-
-    /** */
-    private String cfgPath;
-
-    /** */
-    private String igniteDir;
-
-    /**
-     * @param libsDir Path to directory with user libs.
-     * @param cfgPath Path to config file.
-     */
-    public ResourceController(String libsDir, String cfgPath, String igniteDir) {
-        this.libsDir = libsDir;
-        this.cfgPath = cfgPath;
-        this.igniteDir = igniteDir;
-    }
-
-    /**
-     * @param ignite Ignite jar name.
-     * @return Http response.
-     */
-    @GET
-    @Path(IGNITE_PREFIX + "{ignite-dist}")
-    public Response ignite(@PathParam("ignite-dist") String ignite) {
-        return handleRequest(new File(igniteDir + "/" + ignite), "application/zip-archive", ignite);
-    }
-
-    /**
-     * @param lib user's jar.
-     * @return Http response.
-     */
-    @GET
-    @Path(LIBS_PREFIX + "{lib}")
-    public Response lib(@PathParam("lib") String lib) {
-        return handleRequest(new File(libsDir + "/" + lib), "application/java-archive", lib);
-    }
-
-    /**
-     *
-     * @param cfg Config file.
-     * @return Http response.
-     */
-    @GET
-    @Path(CONFIG_PREFIX + "{cfg}")
-    public Response config(@PathParam("cfg") String cfg) {
-        return handleRequest(new File(cfgPath), "application/xml", cfg);
-    }
-
-    /**
-     * @param cfg Config file.
-     * @return Http response.
-     */
-    @GET
-    @Path(DEFAULT_CONFIG + "{cfg}")
-    public Response defaultConfig(@PathParam("cfg") String cfg) {
-        return handleRequest(Thread.currentThread().getContextClassLoader().getResourceAsStream(cfg),
-            "application/xml", cfg);
-    }
-
-    /**
-     *
-     * @param resource File resource.
-     * @param type Type.
-     * @param attachmentName Attachment name.
-     * @return Http response.
-     */
-    private static Response handleRequest(File resource, String type, String attachmentName) {
-        final Response.ResponseBuilder builder = Response.ok(resource, type);
-        builder.header("Content-Disposition", "attachment; filename=\"" + attachmentName + "\"");
-        return builder.build();
-    }
-
-    /**
-     *
-     * @param resource File resource.
-     * @param type Type.
-     * @param attachmentName Attachment name.
-     * @return Http response.
-     */
-    private static Response handleRequest(InputStream resource, String type, String attachmentName) {
-        final Response.ResponseBuilder builder = Response.ok(resource, type);
-        builder.header("Content-Disposition", "attachment; filename=\"" + attachmentName + "\"");
-        return builder.build();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/639c9128/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceHandler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceHandler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceHandler.java
new file mode 100644
index 0000000..ea883e3
--- /dev/null
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceHandler.java
@@ -0,0 +1,142 @@
+/*
+ * 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.mesos.resource;
+
+import org.eclipse.jetty.server.*;
+import org.eclipse.jetty.server.handler.*;
+
+import javax.servlet.*;
+import javax.servlet.http.*;
+import java.io.*;
+import java.nio.channels.*;
+import java.nio.file.*;
+
+/**
+ * HTTP controller which provides on slave resources.
+ */
+public class ResourceHandler extends AbstractHandler {
+    /** */
+    public static final String IGNITE_PREFIX = "/ignite/";
+
+    /** */
+    public static final String LIBS_PREFIX = "/libs/";
+
+    /** */
+    public static final String CONFIG_PREFIX = "/config/";
+
+    /** */
+    public static final String DEFAULT_CONFIG = CONFIG_PREFIX + "default/";
+
+    /** */
+    private String libsDir;
+
+    /** */
+    private String cfgPath;
+
+    /** */
+    private String igniteDir;
+
+    /**
+     * @param libsDir Directory with user's libs.
+     * @param cfgPath Path to config file.
+     * @param igniteDir Directory with ignites.
+     */
+    public ResourceHandler(String libsDir, String cfgPath, String igniteDir) {
+        this.libsDir = libsDir;
+        this.cfgPath = cfgPath;
+        this.igniteDir = igniteDir;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override public void handle(
+        String url,
+        Request request,
+        HttpServletRequest httpServletRequest,
+        HttpServletResponse response) throws IOException, ServletException {
+
+        String[] path = url.split("/");
+
+        String fileName = path[path.length -1];
+
+        String servicePath = url.substring(0, url.length() - fileName.length());
+
+        switch (servicePath) {
+            case IGNITE_PREFIX:
+                handleRequest(response, "application/zip-archive", igniteDir + "/" + fileName);
+
+                request.setHandled(true);
+                break;
+
+            case LIBS_PREFIX:
+                handleRequest(response, "application/java-archive", libsDir + "/" + fileName);
+
+                request.setHandled(true);
+                break;
+
+            case CONFIG_PREFIX:
+                handleRequest(response, "application/xml", cfgPath);
+
+                request.setHandled(true);
+                break;
+
+            case DEFAULT_CONFIG:
+                handleRequest(response, "application/xml",
+                    Thread.currentThread().getContextClassLoader().getResourceAsStream(fileName),
+                    fileName);
+
+                request.setHandled(true);
+                break;
+        }
+    }
+
+    /**
+     * @param response Http response.
+     * @param type Type.
+     * @param path Path to file.
+     * @throws IOException If failed.
+     */
+    private static void handleRequest(HttpServletResponse response, String type, String path) throws IOException {
+        Path path0 = Paths.get(path);
+
+        response.setContentType(type);
+        response.setHeader("Content-Disposition", "attachment; filename=\"" + path0.getFileName() + "\"");
+
+        try (HttpOutput out = (HttpOutput)response.getOutputStream()) {
+            out.sendContent(FileChannel.open(path0, StandardOpenOption.READ));
+        }
+    }
+
+    /**
+     * @param response Http response.
+     * @param type Type.
+     * @param stream Stream.
+     * @param attachmentName Attachment name.
+     * @throws IOException If failed.
+     */
+    private static void handleRequest(HttpServletResponse response, String type, InputStream stream,
+                                      String attachmentName) throws IOException {
+        response.setContentType(type);
+        response.setHeader("Content-Disposition", "attachment; filename=\"" + attachmentName + "\"");
+
+        try (HttpOutput out = (HttpOutput)response.getOutputStream()) {
+            out.sendContent(stream);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/639c9128/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceProvider.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceProvider.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceProvider.java
index 1b1f615..f02d1bf 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceProvider.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceProvider.java
@@ -22,7 +22,7 @@ import org.apache.ignite.mesos.*;
 import java.io.*;
 import java.util.*;
 
-import static org.apache.ignite.mesos.resource.ResourceController.*;
+import static org.apache.ignite.mesos.resource.ResourceHandler.*;
 
 /**
  * Provides path to user's libs and config file.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/639c9128/modules/mesos/src/main/resources/ignite-default-config.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/resources/ignite-default-config.xml b/modules/mesos/src/main/resources/ignite-default-config.xml
index 9fcce97..2f26398 100644
--- a/modules/mesos/src/main/resources/ignite-default-config.xml
+++ b/modules/mesos/src/main/resources/ignite-default-config.xml
@@ -25,10 +25,10 @@
         <property name="discoverySpi">
             <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
                 <property name="ipFinder">
-                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
-                        <property name="shared" value="true"/>
-                    </bean>
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder"/>
                 </property>
+
+                <property name="joinTimeout" value="60000"/>
             </bean>
         </property>
     </bean>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/639c9128/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java b/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
index 4124331..277e0db 100644
--- a/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
+++ b/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.mesos;
 
 import junit.framework.*;
+import org.apache.ignite.mesos.resource.*;
 import org.apache.mesos.*;
 
 import java.util.*;
@@ -33,27 +34,230 @@ public class IgniteSchedulerSelfTest extends TestCase {
     @Override public void setUp() throws Exception {
         super.setUp();
 
-        //scheduler = new IgniteScheduler();
+        ClusterProperties clustProp = new ClusterProperties();
+
+        scheduler = new IgniteScheduler(clustProp, new ResourceProvider() {
+            @Override public String configName() {
+                return "config.xml";
+            }
+
+            @Override public String igniteUrl() {
+                return "ignite.jar";
+            }
+
+            @Override public String igniteConfigUrl() {
+                return "config.xml";
+            }
+
+            @Override public Collection<String> resourceUrl() {
+                return null;
+            }
+        });
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testHostRegister() throws Exception {
-        //Protos.Offer offer = createOffer("hostname", 4, 1024);
+        Protos.Offer offer = createOffer("hostname", 4, 1024);
+
+        DriverMock mock = new DriverMock();
+
+        scheduler.resourceOffers(mock, Arrays.asList(offer));
+
+        assertNotNull(mock.launchedTask);
+        assertEquals(1, mock.launchedTask.size());
+
+        Protos.TaskInfo taskInfo = mock.launchedTask.iterator().next();
+
+        assertEquals(4.0, resources(taskInfo.getResourcesList(), IgniteScheduler.CPUS));
+        assertEquals(1024.0, resources(taskInfo.getResourcesList(), IgniteScheduler.MEM));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeclineByCpu() throws Exception {
+        Protos.Offer offer = createOffer("hostname", 4, 1024);
+
+        DriverMock mock = new DriverMock();
+
+        ClusterProperties clustProp = new ClusterProperties();
+        clustProp.cpus(2);
+
+        scheduler.setClusterProps(clustProp);
+
+        scheduler.resourceOffers(mock, Arrays.asList(offer));
+
+        assertNotNull(mock.launchedTask);
+        assertEquals(1, mock.launchedTask.size());
+
+        Protos.TaskInfo taskInfo = mock.launchedTask.iterator().next();
+
+        assertEquals(2.0, resources(taskInfo.getResourcesList(), IgniteScheduler.CPUS));
+        assertEquals(1024.0, resources(taskInfo.getResourcesList(), IgniteScheduler.MEM));
+
+        mock.clear();
+
+        scheduler.resourceOffers(mock, Arrays.asList(offer));
+
+        assertNull(mock.launchedTask);
+
+        Protos.OfferID declinedOffer = mock.declinedOffer;
+
+        assertEquals(offer.getId(), declinedOffer);
+    }
+
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeclineByMem() throws Exception {
+        Protos.Offer offer = createOffer("hostname", 4, 1024);
+
+        DriverMock mock = new DriverMock();
+
+        ClusterProperties clustProp = new ClusterProperties();
+        clustProp.memory(512);
+
+        scheduler.setClusterProps(clustProp);
+
+        scheduler.resourceOffers(mock, Arrays.asList(offer));
+
+        assertNotNull(mock.launchedTask);
+        assertEquals(1, mock.launchedTask.size());
+
+        Protos.TaskInfo taskInfo = mock.launchedTask.iterator().next();
+
+        assertEquals(4.0, resources(taskInfo.getResourcesList(), IgniteScheduler.CPUS));
+        assertEquals(512.0, resources(taskInfo.getResourcesList(), IgniteScheduler.MEM));
 
-        //scheduler.resourceOffers(DriverStub.INSTANCE, Lists.);
+        mock.clear();
+
+        scheduler.resourceOffers(mock, Arrays.asList(offer));
+
+        assertNull(mock.launchedTask);
+
+        Protos.OfferID declinedOffer = mock.declinedOffer;
+
+        assertEquals(offer.getId(), declinedOffer);
     }
 
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeclineByMemCpu() throws Exception {
+        Protos.Offer offer = createOffer("hostname", 1, 1024);
+
+        DriverMock mock = new DriverMock();
+
+        ClusterProperties clustProp = new ClusterProperties();
+        clustProp.cpus(4);
+        clustProp.memory(2000);
+
+        scheduler.setClusterProps(clustProp);
+
+        double totalMem = 0, totalCpu = 0;
+
+        for (int i = 0; i < 2; i++) {
+            scheduler.resourceOffers(mock, Arrays.asList(offer));
+
+            assertNotNull(mock.launchedTask);
+            assertEquals(1, mock.launchedTask.size());
+
+            Protos.TaskInfo taskInfo = mock.launchedTask.iterator().next();
+
+            totalCpu += resources(taskInfo.getResourcesList(), IgniteScheduler.CPUS);
+            totalMem += resources(taskInfo.getResourcesList(), IgniteScheduler.MEM);
+
+            mock.clear();
+        }
+
+        assertEquals(2.0, totalCpu);
+        assertEquals(2000.0, totalMem);
+
+        scheduler.resourceOffers(mock, Arrays.asList(offer));
+
+        assertNull(mock.launchedTask);
+
+        Protos.OfferID declinedOffer = mock.declinedOffer;
+
+        assertEquals(offer.getId(), declinedOffer);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeclineByCpuMinRequirements() throws Exception {
+        Protos.Offer offer = createOffer("hostname", 8, 10240);
+
+        DriverMock mock = new DriverMock();
+
+        ClusterProperties clustProp = new ClusterProperties();
+        clustProp.minCpuPerNode(12);
+
+        scheduler.setClusterProps(clustProp);
+
+        scheduler.resourceOffers(mock, Arrays.asList(offer));
+
+        assertNotNull(mock.declinedOffer);
+
+        assertEquals(offer.getId(), mock.declinedOffer);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeclineByMemMinRequirements() throws Exception {
+        Protos.Offer offer = createOffer("hostname", 8, 10240);
+
+        DriverMock mock = new DriverMock();
+
+        ClusterProperties clustProp = new ClusterProperties();
+        clustProp.minMemoryPerNode(15000);
+
+        scheduler.setClusterProps(clustProp);
+
+        scheduler.resourceOffers(mock, Arrays.asList(offer));
+
+        assertNotNull(mock.declinedOffer);
+
+        assertEquals(offer.getId(), mock.declinedOffer);
+    }
+
+
+    /**
+     * @param resourceType Resource type.
+     * @return Value.
+     */
+    private Double resources(List<Protos.Resource> resources, String resourceType) {
+        for (Protos.Resource resource : resources) {
+            if (resource.getName().equals(resourceType))
+                return resource.getScalar().getValue();
+        }
+
+        return null;
+    }
+
+    /**
+     * @param hostname Hostname
+     * @param cpu Cpu count.
+     * @param mem Mem size.
+     * @return Offer.
+     */
     private Protos.Offer createOffer(String hostname, double cpu, double mem) {
         return Protos.Offer.newBuilder()
-            .setSlaveId(Protos.SlaveID.newBuilder().setValue("1").build())
+            .setId(Protos.OfferID.newBuilder().setValue("1"))
+            .setSlaveId(Protos.SlaveID.newBuilder().setValue("1"))
+            .setFrameworkId(Protos.FrameworkID.newBuilder().setValue("1"))
             .setHostname(hostname)
             .addResources(Protos.Resource.newBuilder()
+                .setType(Protos.Value.Type.SCALAR)
                 .setName(IgniteScheduler.CPUS)
                 .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpu).build())
                 .build())
             .addResources(Protos.Resource.newBuilder()
+                .setType(Protos.Value.Type.SCALAR)
                 .setName(IgniteScheduler.MEM)
                 .setScalar(Protos.Value.Scalar.newBuilder().setValue(mem).build())
                 .build())
@@ -63,8 +267,22 @@ public class IgniteSchedulerSelfTest extends TestCase {
     /**
      * No-op implementation.
      */
-    public static class DriverStub implements SchedulerDriver {
-        private static final DriverStub INSTANCE = new DriverStub();
+    public static class DriverMock implements SchedulerDriver {
+        /**
+         *
+         */
+        Collection<Protos.TaskInfo> launchedTask;
+
+        /** */
+        Protos.OfferID declinedOffer;
+
+        /**
+         * Clear launched task.
+         */
+        public void clear() {
+            launchedTask = null;
+            declinedOffer = null;
+        }
 
         /** {@inheritDoc} */
         @Override public Protos.Status start() {
@@ -104,23 +322,31 @@ public class IgniteSchedulerSelfTest extends TestCase {
         /** {@inheritDoc} */
         @Override public Protos.Status launchTasks(Collection<Protos.OfferID> offerIds,
             Collection<Protos.TaskInfo> tasks, Protos.Filters filters) {
+            launchedTask = tasks;
+
             return null;
         }
 
         /** {@inheritDoc} */
         @Override public Protos.Status launchTasks(Collection<Protos.OfferID> offerIds,
             Collection<Protos.TaskInfo> tasks) {
+            launchedTask = tasks;
+
             return null;
         }
 
         /** {@inheritDoc} */
         @Override public Protos.Status launchTasks(Protos.OfferID offerId, Collection<Protos.TaskInfo> tasks,
             Protos.Filters filters) {
+            launchedTask = tasks;
+
             return null;
         }
 
         /** {@inheritDoc} */
         @Override public Protos.Status launchTasks(Protos.OfferID offerId, Collection<Protos.TaskInfo> tasks) {
+            launchedTask = tasks;
+
             return null;
         }
 
@@ -131,11 +357,15 @@ public class IgniteSchedulerSelfTest extends TestCase {
 
         /** {@inheritDoc} */
         @Override public Protos.Status declineOffer(Protos.OfferID offerId, Protos.Filters filters) {
+            declinedOffer = offerId;
+
             return null;
         }
 
         /** {@inheritDoc} */
         @Override public Protos.Status declineOffer(Protos.OfferID offerId) {
+            declinedOffer = offerId;
+
             return null;
         }
 


[34/37] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-sprint-5


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

Branch: refs/heads/ignite-gg-10369
Commit: 43f7aafc0cf7b8db7510e6e3966a713c79d3461d
Parents: 3d77af8 98e392c
Author: ashutak <as...@gridgain.com>
Authored: Thu May 28 19:18:29 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Thu May 28 19:18:29 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../src/main/java/org/apache/ignite/Ignite.java |   8 +-
 .../java/org/apache/ignite/IgniteServices.java  |   5 +-
 .../apache/ignite/internal/IgniteKernal.java    |   4 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |   2 +-
 .../org/apache/ignite/services/Service.java     |   5 +-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  20 +-
 .../service/ClosureServiceClientsNodesTest.java | 245 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 modules/extdata/p2p/pom.xml                     |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |   4 +
 .../query/h2/sql/BaseH2CompareQueryTest.java    |  16 ++
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |   2 +-
 42 files changed, 320 insertions(+), 55 deletions(-)
----------------------------------------------------------------------



[26/37] incubator-ignite git commit: # ignite-857 review

Posted by sb...@apache.org.
# ignite-857 review


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

Branch: refs/heads/ignite-gg-10369
Commit: a36214c77564372916cb345b1a08f8e095276832
Parents: 789106b
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu May 28 17:20:58 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu May 28 17:20:58 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/mesos/IgniteScheduler.java   | 12 ++++++++----
 .../apache/ignite/mesos/IgniteSchedulerSelfTest.java    |  8 +++-----
 2 files changed, 11 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a36214c7/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index e833025..9507642 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -40,14 +40,14 @@ public class IgniteScheduler implements Scheduler {
     /** Default port range. */
     public static final String DEFAULT_PORT = ":47500..47510";
 
-    /** Delimiter to use in IP names. */
+    /** Delimiter char. */
     public static final String DELIM = ",";
 
     /** Logger. */
     private static final Logger log = LoggerFactory.getLogger(IgniteScheduler.class);
 
     /** Mutex. */
-    private static final Object mux = new Object();
+    private final Object mux = new Object();
 
     /** ID generator. */
     private AtomicInteger taskIdGenerator = new AtomicInteger();
@@ -89,7 +89,7 @@ public class IgniteScheduler implements Scheduler {
                 Protos.TaskID taskId = Protos.TaskID.newBuilder()
                     .setValue(Integer.toString(taskIdGenerator.incrementAndGet())).build();
 
-                log.info("Launching task: [{}]", igniteTask);
+                log.info("Launching task: {}", igniteTask);
 
                 // Create task to run.
                 Protos.TaskInfo task = createTask(offer, igniteTask, taskId);
@@ -175,7 +175,9 @@ public class IgniteScheduler implements Scheduler {
     /**
      * @return Address running nodes.
      */
-    protected String getAddress(String address) {
+    private String getAddress(String address) {
+        assert Thread.holdsLock(mux);
+
         if (tasks.isEmpty()) {
             if (address != null && !address.isEmpty())
                 return address + DEFAULT_PORT;
@@ -198,6 +200,8 @@ public class IgniteScheduler implements Scheduler {
      * @return Ignite task description.
      */
     private IgniteTask checkOffer(Protos.Offer offer) {
+        assert Thread.holdsLock(mux);
+
         // Check limit on running nodes.
         if (clusterProps.instances() <= tasks.size())
             return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a36214c7/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java b/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
index 13855b5..d627553 100644
--- a/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
+++ b/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
@@ -341,16 +341,14 @@ public class IgniteSchedulerSelfTest extends TestCase {
      * No-op implementation.
      */
     public static class DriverMock implements SchedulerDriver {
-        /**
-         *
-         */
+        /** */
         Collection<Protos.TaskInfo> launchedTask;
 
         /** */
         Protos.OfferID declinedOffer;
 
         /**
-         * Clear launched task.
+         * Clears launched task.
          */
         public void clear() {
             launchedTask = null;
@@ -463,4 +461,4 @@ public class IgniteSchedulerSelfTest extends TestCase {
             return null;
         }
     }
-}
\ No newline at end of file
+}


[06/37] incubator-ignite git commit: #IGNITE-857 Updated discovery logic.

Posted by sb...@apache.org.
#IGNITE-857 Updated discovery logic.


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

Branch: refs/heads/ignite-gg-10369
Commit: 55c166a64a7f64cccf81ef9ec64cd572c11d94af
Parents: 21a1514
Author: nikolay tikhonov <nt...@gridgain.com>
Authored: Mon May 18 18:42:27 2015 +0300
Committer: nikolay tikhonov <nt...@gridgain.com>
Committed: Mon May 18 18:42:27 2015 +0300

----------------------------------------------------------------------
 modules/mesos/pom.xml                           |   7 +
 .../apache/ignite/mesos/IgniteFramework.java    |  97 +++++++
 .../apache/ignite/mesos/IgniteScheduler.java    | 286 +++++++++++++++++++
 .../org/apache/ignite/mesos/package-info.java   |  22 ++
 .../ignite/messo/IgniteAmazonScheduler.java     |  81 ------
 .../apache/ignite/messo/IgniteFramework.java    | 108 -------
 .../apache/ignite/messo/IgniteScheduler.java    | 243 ----------------
 .../org/apache/ignite/messo/package-info.java   |  22 --
 .../org/apache/ignite/IgniteMesosTestSuite.java |  38 +++
 .../ignite/mesos/IgniteSchedulerSelfTest.java   | 165 +++++++++++
 10 files changed, 615 insertions(+), 454 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55c166a6/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index 4d19b11..ef73c0b 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -39,6 +39,13 @@
             <artifactId>slf4j-api</artifactId>
             <version>1.7.12</version>
         </dependency>
+
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>4.11</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 
     <build>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55c166a6/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
new file mode 100644
index 0000000..5c556a1
--- /dev/null
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
@@ -0,0 +1,97 @@
+/*
+ * 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.mesos;
+
+import com.google.protobuf.*;
+import org.apache.mesos.*;
+
+/**
+ * TODO
+ */
+public class IgniteFramework {
+    /**
+     * @param args Args
+     */
+    public static void main(String[] args) {
+        checkArgs(args);
+
+        final int frameworkFailoverTimeout = 0;
+
+        Protos.FrameworkInfo.Builder frameworkBuilder = Protos.FrameworkInfo.newBuilder()
+            .setName("IgniteFramework")
+            .setUser("") // Have Mesos fill in the current user.
+            .setFailoverTimeout(frameworkFailoverTimeout); // timeout in seconds
+
+        if (System.getenv("MESOS_CHECKPOINT") != null) {
+            System.out.println("Enabling checkpoint for the framework");
+            frameworkBuilder.setCheckpoint(true);
+        }
+
+        // create the scheduler
+        final Scheduler scheduler = new IgniteScheduler();
+
+        // create the driver
+        MesosSchedulerDriver driver;
+        if (System.getenv("MESOS_AUTHENTICATE") != null) {
+            System.out.println("Enabling authentication for the framework");
+
+            if (System.getenv("DEFAULT_PRINCIPAL") == null) {
+                System.err.println("Expecting authentication principal in the environment");
+                System.exit(1);
+            }
+
+            if (System.getenv("DEFAULT_SECRET") == null) {
+                System.err.println("Expecting authentication secret in the environment");
+                System.exit(1);
+            }
+
+            Protos.Credential credential = Protos.Credential.newBuilder()
+                .setPrincipal(System.getenv("DEFAULT_PRINCIPAL"))
+                .setSecret(ByteString.copyFrom(System.getenv("DEFAULT_SECRET").getBytes()))
+                .build();
+
+            frameworkBuilder.setPrincipal(System.getenv("DEFAULT_PRINCIPAL"));
+
+            driver = new MesosSchedulerDriver(scheduler, frameworkBuilder.build(), args[0], credential);
+        }
+        else {
+            frameworkBuilder.setPrincipal("ignite-framework-java");
+
+            driver = new MesosSchedulerDriver(scheduler, frameworkBuilder.build(), args[0]);
+        }
+
+        int status = driver.run() == Protos.Status.DRIVER_STOPPED ? 0 : 1;
+
+        // Ensure that the driver process terminates.
+        driver.stop();
+
+        System.exit(status);
+    }
+
+    /**
+     * Check input arguments.
+     *
+     * @param args Arguments.
+     */
+    private static void checkArgs(String[] args) {
+        if (args.length == 0)
+            throw new IllegalArgumentException("Illegal arguments.");
+
+        // TODO: add more
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55c166a6/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
new file mode 100644
index 0000000..7b5623b
--- /dev/null
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -0,0 +1,286 @@
+/*
+ * 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.mesos;
+
+import org.apache.mesos.*;
+import org.slf4j.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * TODO
+ */
+public class IgniteScheduler implements Scheduler {
+    /** Docker image name. */
+    public static final String IMAGE = "apacheignite/ignite-docker";
+
+    /** Startup sctipt path. */
+    public static final String STARTUP_SCRIPT = "/home/ignite/startup.sh";
+
+    /** Cpus. */
+    public static final String CPUS = "cpus";
+
+    /** Mem. */
+    public static final String MEM = "mem";
+
+    /** Default port range. */
+    public static final String DEFAULT_PORT = ":47500..47510";
+
+    /** Delimiter to use in IP names. */
+    public static final String DELIM = ",";
+
+    /** ID generator. */
+    private AtomicInteger taskIdGenerator = new AtomicInteger();
+
+    /** Logger. */
+    private static final Logger log = LoggerFactory.getLogger(IgniteScheduler.class);
+
+    /** Min of memory required. */
+    public static final int MIN_MEMORY = 256;
+
+    /** Mutex. */
+    private static final Object mux = new Object();
+
+    /** Task on host. */
+    private ConcurrentMap<String, String> tasks = new ConcurrentHashMap<>();
+
+    /** {@inheritDoc} */
+    @Override public void registered(SchedulerDriver schedulerDriver, Protos.FrameworkID frameworkID,
+        Protos.MasterInfo masterInfo) {
+        log.info("registered() master={}:{}, framework={}", masterInfo.getIp(), masterInfo.getPort(), frameworkID);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void reregistered(SchedulerDriver schedulerDriver, Protos.MasterInfo masterInfo) {
+        log.info("reregistered");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void resourceOffers(SchedulerDriver schedulerDriver, List<Protos.Offer> offers) {
+        synchronized (mux) {
+            log.info("resourceOffers() with {} offers", offers.size());
+
+            for (Protos.Offer offer : offers) {
+                Tuple<Double, Double> cpuMem = checkOffer(offer);
+
+                // Decline offer which doesn't match by mem or cpu.
+                if (cpuMem == null) {
+                    schedulerDriver.declineOffer(offer.getId());
+
+                    continue;
+                }
+
+                // Generate a unique task ID.
+                Protos.TaskID taskId = Protos.TaskID.newBuilder()
+                    .setValue(Integer.toString(taskIdGenerator.incrementAndGet())).build();
+
+                log.info("Launching task {}", taskId.getValue());
+
+                // Create task to run.
+                Protos.TaskInfo task = createTask(offer, cpuMem, taskId);
+
+                schedulerDriver.launchTasks(Collections.singletonList(offer.getId()),
+                    Collections.singletonList(task),
+                    Protos.Filters.newBuilder().setRefuseSeconds(1).build());
+
+                tasks.put(taskId.getValue(), offer.getHostname());
+            }
+        }
+    }
+
+    /**
+     * Create Task.
+     *
+     * @param offer Offer.
+     * @param cpuMem Cpu and mem on slave.
+     * @param taskId Task id.
+     * @return Task.
+     */
+    protected Protos.TaskInfo createTask(Protos.Offer offer, Tuple<Double, Double> cpuMem, Protos.TaskID taskId) {
+        // Docker image info.
+        Protos.ContainerInfo.DockerInfo.Builder docker = Protos.ContainerInfo.DockerInfo.newBuilder()
+            .setImage(IMAGE)
+            .setNetwork(Protos.ContainerInfo.DockerInfo.Network.HOST);
+
+        // Container info.
+        Protos.ContainerInfo.Builder cont = Protos.ContainerInfo.newBuilder();
+        cont.setType(Protos.ContainerInfo.Type.DOCKER);
+        cont.setDocker(docker.build());
+
+        return Protos.TaskInfo.newBuilder()
+            .setName("task " + taskId.getValue())
+            .setTaskId(taskId)
+            .setSlaveId(offer.getSlaveId())
+            .addResources(Protos.Resource.newBuilder()
+                .setName(CPUS)
+                .setType(Protos.Value.Type.SCALAR)
+                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem.get1())))
+            .addResources(Protos.Resource.newBuilder()
+                .setName(MEM)
+                .setType(Protos.Value.Type.SCALAR)
+                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem.get2())))
+            .setContainer(cont)
+            .setCommand(Protos.CommandInfo.newBuilder()
+                .setShell(false)
+                .addArguments(STARTUP_SCRIPT)
+                .addArguments(String.valueOf(cpuMem.get2().intValue()))
+                .addArguments(getAddress()))
+            .build();
+    }
+
+    /**
+     * @return Address running nodes.
+     */
+    protected String getAddress() {
+        if (tasks.isEmpty())
+            return "";
+
+        StringBuilder sb = new StringBuilder();
+
+        for (String host : tasks.values())
+            sb.append(host).append(DEFAULT_PORT).append(DELIM);
+
+        return sb.substring(0, sb.length() - 1);
+    }
+
+    /**
+     * Check slave resources and return resources infos.
+     *
+     * @param offer Offer request.
+     * @return Pair where first is cpus, second is memory.
+     */
+    private Tuple<Double, Double> checkOffer(Protos.Offer offer) {
+        double cpus = -1;
+        double mem = -1;
+
+        for (Protos.Resource resource : offer.getResourcesList()) {
+            if (resource.getName().equals(CPUS)) {
+                if (resource.getType().equals(Protos.Value.Type.SCALAR))
+                    cpus = resource.getScalar().getValue();
+                else
+                    log.debug("Cpus resource was not a scalar: " + resource.getType().toString());
+            }
+            else if (resource.getName().equals(MEM)) {
+                if (resource.getType().equals(Protos.Value.Type.SCALAR))
+                    mem = resource.getScalar().getValue();
+                else
+                    log.debug("Mem resource was not a scalar: " + resource.getType().toString());
+            }
+            else if (resource.getName().equals("disk"))
+                log.debug("Ignoring disk resources from offer");
+        }
+
+        if (cpus < 0)
+            log.debug("No cpus resource present");
+        if (mem < 0)
+            log.debug("No mem resource present");
+
+        if (cpus >= 1 && MIN_MEMORY <= mem)
+            return new Tuple<>(cpus, mem);
+        else {
+            log.info("Offer not sufficient for slave request:\n" + offer.getResourcesList().toString() +
+                "\n" + offer.getAttributesList().toString() +
+                "\nRequested for slave:\n" +
+                "  cpus:  " + cpus + "\n" +
+                "  mem:   " + mem);
+
+            return null;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void offerRescinded(SchedulerDriver schedulerDriver, Protos.OfferID offerID) {
+        log.info("offerRescinded()");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void statusUpdate(SchedulerDriver schedulerDriver, Protos.TaskStatus taskStatus) {
+        final String taskId = taskStatus.getTaskId().getValue();
+
+        log.info("statusUpdate() task {} is in state {}", taskId, taskStatus.getState());
+
+        switch (taskStatus.getState()) {
+            case TASK_FAILED:
+            case TASK_FINISHED:
+                tasks.remove(taskId);
+                break;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void frameworkMessage(SchedulerDriver schedulerDriver, Protos.ExecutorID executorID,
+        Protos.SlaveID slaveID, byte[] bytes) {
+        log.info("frameworkMessage()");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void disconnected(SchedulerDriver schedulerDriver) {
+        log.info("disconnected()");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void slaveLost(SchedulerDriver schedulerDriver, Protos.SlaveID slaveID) {
+        log.info("slaveLost()");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void executorLost(SchedulerDriver schedulerDriver, Protos.ExecutorID executorID,
+        Protos.SlaveID slaveID, int i) {
+        log.info("executorLost()");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void error(SchedulerDriver schedulerDriver, String s) {
+        log.error("error() {}", s);
+    }
+
+    /**
+     * Tuple.
+     */
+    public static class Tuple<A, B> {
+        /** */
+        private final A val1;
+
+        /** */
+        private final B val2;
+
+        /**
+         *
+         */
+        public Tuple(A val1, B val2) {
+            this.val1 = val1;
+            this.val2 = val2;
+        }
+
+        /**
+         * @return val1
+         */
+        public A get1() {
+            return val1;
+        }
+
+        /**
+         * @return val2
+         */
+        public B get2() {
+            return val2;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55c166a6/modules/mesos/src/main/java/org/apache/ignite/mesos/package-info.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/package-info.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/package-info.java
new file mode 100644
index 0000000..49ddf86
--- /dev/null
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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 description. -->
+ * Messo Framework.
+ */
+package org.apache.ignite.mesos;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55c166a6/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteAmazonScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteAmazonScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteAmazonScheduler.java
deleted file mode 100644
index b11e7c6..0000000
--- a/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteAmazonScheduler.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * 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.messo;
-
-import org.apache.mesos.*;
-
-/**
- * TODO
- */
-public class IgniteAmazonScheduler extends IgniteScheduler {
-    /** */
-    public static final String AMAZON = "amazon";
-
-    /** Amazon credential. */
-    private final String accessKey, secretKey;
-
-    /**
-     * Constructor.
-     *
-     * @param accessKey Access key.
-     * @param secretKey Secret key.
-     */
-    public IgniteAmazonScheduler(String accessKey, String secretKey) {
-        assert accessKey != null;
-        assert secretKey != null;
-
-        this.accessKey = accessKey;
-        this.secretKey = secretKey;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Protos.TaskInfo createTask(Protos.Offer offer, Tuple<Double, Double> cpuMem,
-        Protos.TaskID taskId) {
-        // Docker image info.
-        Protos.ContainerInfo.DockerInfo.Builder docker = Protos.ContainerInfo.DockerInfo.newBuilder()
-            .setImage(IMAGE)
-            .setNetwork(Protos.ContainerInfo.DockerInfo.Network.HOST);
-
-        // Container info.
-        Protos.ContainerInfo.Builder cont = Protos.ContainerInfo.newBuilder();
-        cont.setType(Protos.ContainerInfo.Type.DOCKER);
-        cont.setDocker(docker.build());
-
-        return Protos.TaskInfo.newBuilder()
-            .setName("task " + taskId.getValue())
-            .setTaskId(taskId)
-            .setSlaveId(offer.getSlaveId())
-            .addResources(Protos.Resource.newBuilder()
-                .setName(CPUS)
-                .setType(Protos.Value.Type.SCALAR)
-                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem.get1())))
-            .addResources(Protos.Resource.newBuilder()
-                .setName(MEM)
-                .setType(Protos.Value.Type.SCALAR)
-                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem.get2())))
-            .setContainer(cont)
-            .setCommand(Protos.CommandInfo.newBuilder()
-                .setShell(false)
-                .addArguments(STARTUP_SCRIPT)
-                .addArguments(String.valueOf(cpuMem.get2().intValue()))
-                .addArguments(AMAZON)
-                .addArguments(accessKey)
-                .addArguments(secretKey))
-            .build();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55c166a6/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteFramework.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteFramework.java b/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteFramework.java
deleted file mode 100644
index dfc3eb2..0000000
--- a/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteFramework.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * 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.messo;
-
-import com.google.protobuf.*;
-import org.apache.mesos.*;
-
-/**
- * TODO
- */
-public class IgniteFramework {
-    /**
-     * @param args Args
-     */
-    public static void main(String[] args) {
-        checkArgs(args);
-
-        final int frameworkFailoverTimeout = 0;
-
-        Protos.FrameworkInfo.Builder frameworkBuilder = Protos.FrameworkInfo.newBuilder()
-            .setName("IgniteFramework")
-            .setUser("") // Have Mesos fill in the current user.
-            .setFailoverTimeout(frameworkFailoverTimeout); // timeout in seconds
-
-        if (System.getenv("MESOS_CHECKPOINT") != null) {
-            System.out.println("Enabling checkpoint for the framework");
-            frameworkBuilder.setCheckpoint(true);
-        }
-
-        // create the scheduler
-        final Scheduler scheduler = createIgniteScheduler(args);
-
-        // create the driver
-        MesosSchedulerDriver driver;
-        if (System.getenv("MESOS_AUTHENTICATE") != null) {
-            System.out.println("Enabling authentication for the framework");
-
-            if (System.getenv("DEFAULT_PRINCIPAL") == null) {
-                System.err.println("Expecting authentication principal in the environment");
-                System.exit(1);
-            }
-
-            if (System.getenv("DEFAULT_SECRET") == null) {
-                System.err.println("Expecting authentication secret in the environment");
-                System.exit(1);
-            }
-
-            Protos.Credential credential = Protos.Credential.newBuilder()
-                .setPrincipal(System.getenv("DEFAULT_PRINCIPAL"))
-                .setSecret(ByteString.copyFrom(System.getenv("DEFAULT_SECRET").getBytes()))
-                .build();
-
-            frameworkBuilder.setPrincipal(System.getenv("DEFAULT_PRINCIPAL"));
-
-            driver = new MesosSchedulerDriver(scheduler, frameworkBuilder.build(), args[0], credential);
-        }
-        else {
-            frameworkBuilder.setPrincipal("ignite-framework-java");
-
-            driver = new MesosSchedulerDriver(scheduler, frameworkBuilder.build(), args[0]);
-        }
-
-        int status = driver.run() == Protos.Status.DRIVER_STOPPED ? 0 : 1;
-
-        // Ensure that the driver process terminates.
-        driver.stop();
-
-        System.exit(status);
-    }
-
-    /**
-     * @param args Arguments.
-     * @return Ignite scheduler.
-     */
-    private static IgniteScheduler createIgniteScheduler(String args[]) {
-        if (args.length >= 3 && args[1].equals(IgniteAmazonScheduler.AMAZON))
-            return new IgniteAmazonScheduler(args[2], args[3]);
-        else
-            return new IgniteScheduler();
-    }
-
-    /**
-     * Check input arguments.
-     *
-     * @param args Arguments.
-     */
-    private static void checkArgs(String[] args) {
-        if (args.length == 0)
-            throw new IllegalArgumentException("Illegal arguments.");
-
-        // TODO: add more
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55c166a6/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java
deleted file mode 100644
index c8b577f..0000000
--- a/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java
+++ /dev/null
@@ -1,243 +0,0 @@
-/*
- * 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.messo;
-
-import org.apache.mesos.*;
-import org.slf4j.*;
-
-import java.util.*;
-import java.util.concurrent.atomic.*;
-
-/**
- * TODO
- */
-public class IgniteScheduler implements Scheduler {
-    /** Docker image name. */
-    public static final String IMAGE = "apacheignite/ignite-docker";
-
-    /** Startup sctipt path. */
-    public static final String STARTUP_SCRIPT = "/home/ignite/startup.sh";
-
-    /** Cpus. */
-    public static final String CPUS = "cpus";
-
-    /** Mem. */
-    public static final String MEM = "mem";
-
-    /** ID generator. */
-    private AtomicInteger taskIdGenerator = new AtomicInteger();
-
-    /** Logger. */
-    private static final Logger log = LoggerFactory.getLogger(IgniteScheduler.class);
-
-    /** Min of memory required. */
-    public static final int MIN_MEMORY = 256;
-
-    /** {@inheritDoc} */
-    @Override public void registered(SchedulerDriver schedulerDriver, Protos.FrameworkID frameworkID,
-        Protos.MasterInfo masterInfo) {
-        log.info("registered() master={}:{}, framework={}", masterInfo.getIp(), masterInfo.getPort(), frameworkID);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void reregistered(SchedulerDriver schedulerDriver, Protos.MasterInfo masterInfo) {
-        log.info("reregistered");
-    }
-
-    /** {@inheritDoc} */
-    @Override public void resourceOffers(SchedulerDriver schedulerDriver, List<Protos.Offer> offers) {
-        log.info("resourceOffers() with {} offers", offers.size());
-
-        for (Protos.Offer offer : offers) {
-            Tuple<Double, Double> cpuMem = checkOffer(offer);
-
-            // Decline offer which doesn't match by mem or cpu.
-            if (cpuMem == null) {
-                schedulerDriver.declineOffer(offer.getId());
-
-                continue;
-            }
-
-            // Generate a unique task ID.
-            Protos.TaskID taskId = Protos.TaskID.newBuilder()
-                .setValue(Integer.toString(taskIdGenerator.incrementAndGet())).build();
-
-            log.info("Launching task {}", taskId.getValue());
-
-            // Create task to run.
-            Protos.TaskInfo task = createTask(offer, cpuMem, taskId);
-
-            schedulerDriver.launchTasks(Collections.singletonList(offer.getId()),
-                Collections.singletonList(task),
-                Protos.Filters.newBuilder().setRefuseSeconds(1).build());
-        }
-    }
-
-    /**
-     * Create Task.
-     * @param offer Offer.
-     * @param cpuMem Cpu and mem on slave.
-     * @param taskId Task id.
-     * @return Task.
-     */
-    protected Protos.TaskInfo createTask(Protos.Offer offer, Tuple<Double, Double> cpuMem, Protos.TaskID taskId) {
-        // Docker image info.
-        Protos.ContainerInfo.DockerInfo.Builder docker = Protos.ContainerInfo.DockerInfo.newBuilder()
-            .setImage(IMAGE)
-            .setNetwork(Protos.ContainerInfo.DockerInfo.Network.HOST);
-
-        // Container info.
-        Protos.ContainerInfo.Builder cont = Protos.ContainerInfo.newBuilder();
-        cont.setType(Protos.ContainerInfo.Type.DOCKER);
-        cont.setDocker(docker.build());
-
-        return Protos.TaskInfo.newBuilder()
-            .setName("task " + taskId.getValue())
-            .setTaskId(taskId)
-            .setSlaveId(offer.getSlaveId())
-            .addResources(Protos.Resource.newBuilder()
-                .setName(CPUS)
-                .setType(Protos.Value.Type.SCALAR)
-                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem.get2())))
-            .addResources(Protos.Resource.newBuilder()
-                .setName(MEM)
-                .setType(Protos.Value.Type.SCALAR)
-                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem.get2())))
-            .setContainer(cont)
-            .setCommand(Protos.CommandInfo.newBuilder()
-                .setShell(false)
-                .addArguments(STARTUP_SCRIPT)
-                .addArguments(String.valueOf(cpuMem.get2().intValue())))
-            .build();
-    }
-
-    /**
-     * Check slave resources and return resources infos.
-     *
-     * @param offer Offer request.
-     * @return Pair where first is cpus, second is memory.
-     */
-    private Tuple<Double, Double> checkOffer(Protos.Offer offer) {
-        double cpus = -1;
-        double mem = -1;
-
-        for (Protos.Resource resource : offer.getResourcesList()) {
-            if (resource.getName().equals(CPUS)) {
-                if (resource.getType().equals(Protos.Value.Type.SCALAR))
-                    cpus = resource.getScalar().getValue();
-                else
-                    log.debug("Cpus resource was not a scalar: " + resource.getType().toString());
-            }
-            else if (resource.getName().equals(MEM)) {
-                if (resource.getType().equals(Protos.Value.Type.SCALAR))
-                    mem = resource.getScalar().getValue();
-                else
-                    log.debug("Mem resource was not a scalar: " + resource.getType().toString());
-            }
-            else if (resource.getName().equals("disk"))
-                log.debug("Ignoring disk resources from offer");
-        }
-
-        if (cpus < 0)
-            log.debug("No cpus resource present");
-        if (mem < 0)
-            log.debug("No mem resource present");
-
-        if (cpus >= 1 && MIN_MEMORY <= mem)
-            return new Tuple<>(cpus, mem);
-        else {
-            log.info("Offer not sufficient for slave request:\n" + offer.getResourcesList().toString() +
-                "\n" + offer.getAttributesList().toString() +
-                "\nRequested for slave:\n" +
-                "  cpus:  " + cpus + "\n" +
-                "  mem:   " + mem);
-
-            return null;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void offerRescinded(SchedulerDriver schedulerDriver, Protos.OfferID offerID) {
-        log.info("offerRescinded()");
-    }
-
-    /** {@inheritDoc} */
-    @Override public void statusUpdate(SchedulerDriver schedulerDriver, Protos.TaskStatus taskStatus) {
-        log.info("statusUpdate() task {} ", taskStatus);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void frameworkMessage(SchedulerDriver schedulerDriver, Protos.ExecutorID executorID,
-        Protos.SlaveID slaveID, byte[] bytes) {
-        log.info("frameworkMessage()");
-    }
-
-    /** {@inheritDoc} */
-    @Override public void disconnected(SchedulerDriver schedulerDriver) {
-        log.info("disconnected()");
-    }
-
-    /** {@inheritDoc} */
-    @Override public void slaveLost(SchedulerDriver schedulerDriver, Protos.SlaveID slaveID) {
-        log.info("slaveLost()");
-    }
-
-    /** {@inheritDoc} */
-    @Override public void executorLost(SchedulerDriver schedulerDriver, Protos.ExecutorID executorID,
-        Protos.SlaveID slaveID, int i) {
-        log.info("executorLost()");
-    }
-
-    /** {@inheritDoc} */
-    @Override public void error(SchedulerDriver schedulerDriver, String s) {
-        log.error("error() {}", s);
-    }
-
-    /**
-     * Tuple.
-     */
-    public static class Tuple<A, B> {
-        /** */
-        private final A val1;
-
-        /** */
-        private final B val2;
-
-        /**
-         *
-         */
-        public Tuple(A val1, B val2) {
-            this.val1 = val1;
-            this.val2 = val2;
-        }
-
-        /**
-         * @return val1
-         */
-        public A get1() {
-            return val1;
-        }
-
-        /**
-         * @return val2
-         */
-        public B get2() {
-            return val2;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55c166a6/modules/mesos/src/main/java/org/apache/ignite/messo/package-info.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/messo/package-info.java b/modules/mesos/src/main/java/org/apache/ignite/messo/package-info.java
deleted file mode 100644
index c48ca38..0000000
--- a/modules/mesos/src/main/java/org/apache/ignite/messo/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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 description. -->
- * Messo Framework.
- */
-package org.apache.ignite.messo;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55c166a6/modules/mesos/src/test/java/org/apache/ignite/IgniteMesosTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/test/java/org/apache/ignite/IgniteMesosTestSuite.java b/modules/mesos/src/test/java/org/apache/ignite/IgniteMesosTestSuite.java
new file mode 100644
index 0000000..f1bcb90
--- /dev/null
+++ b/modules/mesos/src/test/java/org/apache/ignite/IgniteMesosTestSuite.java
@@ -0,0 +1,38 @@
+/*
+ * 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;
+
+import junit.framework.*;
+import org.apache.ignite.mesos.*;
+
+/**
+ * Apache Mesos integration tests.
+ */
+public class IgniteMesosTestSuite extends TestSuite {
+    /**
+     * @return Test suite.
+     * @throws Exception Thrown in case of the failure.
+     */
+    public static TestSuite suite() throws Exception {
+        TestSuite suite = new TestSuite("Apache Mesos Integration Test Suite");
+
+        suite.addTest(new TestSuite(IgniteSchedulerSelfTest.class));
+
+        return suite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55c166a6/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java b/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
new file mode 100644
index 0000000..5534b2c
--- /dev/null
+++ b/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
@@ -0,0 +1,165 @@
+/*
+ * 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.mesos;
+
+import junit.framework.*;
+import org.apache.mesos.*;
+
+import java.util.*;
+
+/**
+ * Scheduler tests.
+ */
+public class IgniteSchedulerSelfTest extends TestCase {
+    /** */
+    private IgniteScheduler scheduler;
+
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        super.setUp();
+
+        scheduler = new IgniteScheduler();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testHostRegister() throws Exception {
+        //Protos.Offer offer = createOffer("hostname", 4, 1024);
+
+        //scheduler.resourceOffers(DriverStub.INSTANCE, Lists.);
+    }
+
+    private Protos.Offer createOffer(String hostname, double cpu, double mem) {
+        return Protos.Offer.newBuilder()
+            .setSlaveId(Protos.SlaveID.newBuilder().setValue("1").build())
+            .setHostname(hostname)
+            .addResources(Protos.Resource.newBuilder()
+                .setName(IgniteScheduler.CPUS)
+                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpu).build())
+                .build())
+            .addResources(Protos.Resource.newBuilder()
+                .setName(IgniteScheduler.MEM)
+                .setScalar(Protos.Value.Scalar.newBuilder().setValue(mem).build())
+                .build())
+            .build();
+    }
+
+    /**
+     * No-op implementation.
+     */
+    public static class DriverStub implements SchedulerDriver {
+        private static final DriverStub INSTANCE = new DriverStub();
+
+        /** {@inheritDoc} */
+        @Override public Protos.Status start() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Protos.Status stop(boolean failover) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Protos.Status stop() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Protos.Status abort() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Protos.Status join() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Protos.Status run() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Protos.Status requestResources(Collection<Protos.Request> requests) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Protos.Status launchTasks(Collection<Protos.OfferID> offerIds,
+            Collection<Protos.TaskInfo> tasks, Protos.Filters filters) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Protos.Status launchTasks(Collection<Protos.OfferID> offerIds,
+            Collection<Protos.TaskInfo> tasks) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Protos.Status launchTasks(Protos.OfferID offerId, Collection<Protos.TaskInfo> tasks,
+            Protos.Filters filters) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Protos.Status launchTasks(Protos.OfferID offerId, Collection<Protos.TaskInfo> tasks) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Protos.Status killTask(Protos.TaskID taskId) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Protos.Status declineOffer(Protos.OfferID offerId, Protos.Filters filters) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Protos.Status declineOffer(Protos.OfferID offerId) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Protos.Status reviveOffers() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Protos.Status acknowledgeStatusUpdate(Protos.TaskStatus status) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Protos.Status sendFrameworkMessage(Protos.ExecutorID executorId, Protos.SlaveID slaveId,
+            byte[] data) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Protos.Status reconcileTasks(Collection<Protos.TaskStatus> statuses) {
+            return null;
+        }
+
+
+    }
+}
\ No newline at end of file



[13/37] incubator-ignite git commit: #IGNITE-857 WIP.

Posted by sb...@apache.org.
#IGNITE-857 WIP.


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

Branch: refs/heads/ignite-gg-10369
Commit: 5e3bcb2e423df46a325834de236b3a8a91ff5659
Parents: daac03e
Author: nikolay tikhonov <nt...@gridgain.com>
Authored: Tue May 26 09:14:21 2015 +0300
Committer: nikolay tikhonov <nt...@gridgain.com>
Committed: Tue May 26 09:14:21 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/mesos/IgniteScheduler.java    | 67 +++++++++-----------
 .../ignite/mesos/IgniteSchedulerSelfTest.java   |  1 -
 2 files changed, 30 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5e3bcb2e/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index cd517fb..7d713cd 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -71,17 +71,6 @@ public class IgniteScheduler implements Scheduler {
     }
 
     /** {@inheritDoc} */
-    @Override public void registered(SchedulerDriver schedulerDriver, Protos.FrameworkID frameworkID,
-        Protos.MasterInfo masterInfo) {
-        log.info("registered() master={}:{}, framework={}", masterInfo.getIp(), masterInfo.getPort(), frameworkID);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void reregistered(SchedulerDriver schedulerDriver, Protos.MasterInfo masterInfo) {
-        log.info("reregistered()");
-    }
-
-    /** {@inheritDoc} */
     @Override public void resourceOffers(SchedulerDriver schedulerDriver, List<Protos.Offer> offers) {
         synchronized (mux) {
             log.info("resourceOffers() with {} offers", offers.size());
@@ -201,28 +190,24 @@ public class IgniteScheduler implements Scheduler {
                 if (resource.getType().equals(Protos.Value.Type.SCALAR))
                     cpus = resource.getScalar().getValue();
                 else
-                    log.debug("Cpus resource was not a scalar: " + resource.getType().toString());
+                    log.debug("Cpus resource was not a scalar: {}" + resource.getType());
             }
             else if (resource.getName().equals(MEM)) {
                 if (resource.getType().equals(Protos.Value.Type.SCALAR))
                     mem = resource.getScalar().getValue();
                 else
-                    log.debug("Mem resource was not a scalar: " + resource.getType().toString());
+                    log.debug("Mem resource was not a scalar: {}", resource.getType());
             }
             else if (resource.getName().equals(DISK))
                 if (resource.getType().equals(Protos.Value.Type.SCALAR))
                     disk = resource.getScalar().getValue();
                 else
-                    log.debug("Disk resource was not a scalar: " + resource.getType().toString());
+                    log.debug("Disk resource was not a scalar: {}", resource.getType());
         }
 
         // Check that slave satisfies min requirements.
         if (cpus < clusterLimit.minCpuPerNode()  && mem < clusterLimit.minMemoryPerNode() ) {
-            log.info("Offer not sufficient for slave request:\n" + offer.getResourcesList().toString() +
-                "\n" + offer.getAttributesList().toString() +
-                "\nRequested for slave:\n" +
-                "  cpus:  " + cpus + "\n" +
-                "  mem:   " + mem);
+            log.debug("Offer not sufficient for slave request: {}", offer.getResourcesList());
 
             return null;
         }
@@ -245,22 +230,13 @@ public class IgniteScheduler implements Scheduler {
         if (cpus > 0 && mem > 0)
             return new IgniteTask(offer.getHostname(), cpus, mem, disk);
         else {
-            log.info("Offer not sufficient for slave request:\n" + offer.getResourcesList().toString() +
-                "\n" + offer.getAttributesList().toString() +
-                "\nRequested for slave:\n" +
-                "  cpus:  " + cpus + "\n" +
-                "  mem:   " + mem);
+            log.debug("Offer not sufficient for slave request: {}", offer.getResourcesList());
 
             return null;
         }
     }
 
     /** {@inheritDoc} */
-    @Override public void offerRescinded(SchedulerDriver schedulerDriver, Protos.OfferID offerID) {
-        log.info("offerRescinded()");
-    }
-
-    /** {@inheritDoc} */
     @Override public void statusUpdate(SchedulerDriver schedulerDriver, Protos.TaskStatus taskStatus) {
         final String taskId = taskStatus.getTaskId().getValue();
 
@@ -296,29 +272,46 @@ public class IgniteScheduler implements Scheduler {
     }
 
     /** {@inheritDoc} */
-    @Override public void frameworkMessage(SchedulerDriver schedulerDriver, Protos.ExecutorID executorID,
-        Protos.SlaveID slaveID, byte[] bytes) {
-        log.info("frameworkMessage()");
+    @Override public void registered(SchedulerDriver schedulerDriver, Protos.FrameworkID frameworkID,
+        Protos.MasterInfo masterInfo) {
+        log.info("Scheduler registered. Master: [{}:{}], framework=[{}]", masterInfo.getIp(), masterInfo.getPort(),
+            frameworkID);
     }
 
     /** {@inheritDoc} */
     @Override public void disconnected(SchedulerDriver schedulerDriver) {
-        log.info("disconnected()");
+        log.info("Scheduler disconnected.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void error(SchedulerDriver schedulerDriver, String s) {
+        log.error("Failed. Error message: {}", s);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void frameworkMessage(SchedulerDriver schedulerDriver, Protos.ExecutorID executorID,
+        Protos.SlaveID slaveID, byte[] bytes) {
+        // No-op.
     }
 
     /** {@inheritDoc} */
     @Override public void slaveLost(SchedulerDriver schedulerDriver, Protos.SlaveID slaveID) {
-        log.info("slaveLost()");
+        // No-op.
     }
 
     /** {@inheritDoc} */
     @Override public void executorLost(SchedulerDriver schedulerDriver, Protos.ExecutorID executorID,
         Protos.SlaveID slaveID, int i) {
-        log.info("executorLost()");
+        // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public void error(SchedulerDriver schedulerDriver, String s) {
-        log.error("error() {}", s);
+    @Override public void offerRescinded(SchedulerDriver schedulerDriver, Protos.OfferID offerID) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void reregistered(SchedulerDriver schedulerDriver, Protos.MasterInfo masterInfo) {
+        // No-op.
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5e3bcb2e/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java b/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
index 8f8ca8b..4124331 100644
--- a/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
+++ b/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
@@ -159,6 +159,5 @@ public class IgniteSchedulerSelfTest extends TestCase {
         @Override public Protos.Status reconcileTasks(Collection<Protos.TaskStatus> statuses) {
             return null;
         }
-
     }
 }
\ No newline at end of file


[18/37] incubator-ignite git commit: #IGNITE-857 Added constraint on host. Added test.

Posted by sb...@apache.org.
#IGNITE-857 Added constraint on host. Added test.


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

Branch: refs/heads/ignite-gg-10369
Commit: 9a951e0ccdd63295632babc48519bc672c0ad0e2
Parents: 9bed1ff
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed May 27 16:14:55 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed May 27 16:14:55 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/mesos/ClusterProperties.java  | 53 ++++++++++++++++----
 .../apache/ignite/mesos/IgniteScheduler.java    |  5 ++
 .../ignite/mesos/IgniteSchedulerSelfTest.java   | 45 +++++++++++++----
 3 files changed, 85 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9a951e0c/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
index 25216ca..9f0b304 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
@@ -17,14 +17,20 @@
 
 package org.apache.ignite.mesos;
 
+import org.slf4j.*;
+
 import java.io.*;
 import java.net.*;
 import java.util.*;
+import java.util.regex.*;
 
 /**
  * Cluster settings.
  */
 public class ClusterProperties {
+    /** */
+    private static final Logger log = LoggerFactory.getLogger(ClusterProperties.class);
+
     /** Unlimited. */
     public static final double UNLIMITED = Double.MAX_VALUE;
 
@@ -170,6 +176,12 @@ public class ClusterProperties {
     private String igniteCfgUrl = null;
 
     /** */
+    public static final String IGNITE_HOSTNAME_CONSTRAINT = "IGNITE_HOSTNAME_CONSTRAINT";
+
+    /** Url to ignite config. */
+    private Pattern hostnameConstraint = null;
+
+    /** */
     public ClusterProperties() {
         // No-op.
     }
@@ -184,21 +196,21 @@ public class ClusterProperties {
     /**
      * @return CPU count limit.
      */
-    public double cpus(){
+    public double cpus() {
         return cpu;
     }
 
     /**
      * Set CPU count limit.
      */
-    public void cpus(double cpu){
+    public void cpus(double cpu) {
         this.cpu = cpu;
     }
 
     /**
      * @return CPU count limit.
      */
-    public double cpusPerNode(){
+    public double cpusPerNode() {
         return cpuPerNode;
     }
 
@@ -216,7 +228,6 @@ public class ClusterProperties {
         this.mem = mem;
     }
 
-
     /**
      * @return mem limit.
      */
@@ -262,6 +273,15 @@ public class ClusterProperties {
     }
 
     /**
+     * Sets hostname constraint.
+     *
+     * @param pattern Hostname pattern.
+     */
+    public void hostnameConstraint(Pattern pattern) {
+        this.hostnameConstraint = pattern;
+    }
+
+    /**
      * @return min cpu count per node.
      */
     public double minCpuPerNode() {
@@ -348,6 +368,13 @@ public class ClusterProperties {
     }
 
     /**
+     * @return Host name constraint.
+     */
+    public Pattern hostnameConstraint() {
+        return hostnameConstraint;
+    }
+
+    /**
      * @param config path to config file.
      * @return Cluster configuration.
      */
@@ -396,6 +423,17 @@ public class ClusterProperties {
             prop.igniteCfg = getStringProperty(IGNITE_CONFIG_XML, props, null);
             prop.userLibs = getStringProperty(IGNITE_USERS_LIBS, props, null);
 
+            String pattern = getStringProperty(IGNITE_HOSTNAME_CONSTRAINT, props, null);
+
+            if (pattern != null) {
+                try {
+                    prop.hostnameConstraint = Pattern.compile(pattern);
+                }
+                catch (PatternSyntaxException e) {
+                    log.warn("IGNITE_HOSTNAME_CONSTRAINT has invalid pattern. It will be ignore.", e);
+                }
+            }
+
             return prop;
         }
         catch (IOException e) {
@@ -440,11 +478,8 @@ public class ClusterProperties {
     /**
      * Finds a local, non-loopback, IPv4 address
      *
-     * @return The first non-loopback IPv4 address found, or
-     *         <code>null</code> if no such addresses found
-     * @throws java.net.SocketException
-     *            If there was a problem querying the network
-     *            interfaces
+     * @return The first non-loopback IPv4 address found, or <code>null</code> if no such addresses found
+     * @throws java.net.SocketException If there was a problem querying the network interfaces
      */
     public static String getNonLoopbackAddress() throws SocketException {
         Enumeration<NetworkInterface> ifaces = NetworkInterface.getNetworkInterfaces();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9a951e0c/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index 9041629..9b8553e 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -203,6 +203,11 @@ public class IgniteScheduler implements Scheduler {
         double mem = -1;
         double disk = -1;
 
+        // Check host name
+        if (clusterProps.hostnameConstraint() != null
+            && clusterProps.hostnameConstraint().matcher(offer.getHostname()).matches())
+            return null;
+
         // Collect resource on slave.
         for (Protos.Resource resource : offer.getResourcesList()) {
             if (resource.getName().equals(CPUS)) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9a951e0c/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java b/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
index 277e0db..337b47c 100644
--- a/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
+++ b/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.mesos.resource.*;
 import org.apache.mesos.*;
 
 import java.util.*;
+import java.util.regex.*;
 
 /**
  * Scheduler tests.
@@ -63,7 +64,7 @@ public class IgniteSchedulerSelfTest extends TestCase {
 
         DriverMock mock = new DriverMock();
 
-        scheduler.resourceOffers(mock, Arrays.asList(offer));
+        scheduler.resourceOffers(mock, Collections.singletonList(offer));
 
         assertNotNull(mock.launchedTask);
         assertEquals(1, mock.launchedTask.size());
@@ -87,7 +88,7 @@ public class IgniteSchedulerSelfTest extends TestCase {
 
         scheduler.setClusterProps(clustProp);
 
-        scheduler.resourceOffers(mock, Arrays.asList(offer));
+        scheduler.resourceOffers(mock, Collections.singletonList(offer));
 
         assertNotNull(mock.launchedTask);
         assertEquals(1, mock.launchedTask.size());
@@ -99,7 +100,7 @@ public class IgniteSchedulerSelfTest extends TestCase {
 
         mock.clear();
 
-        scheduler.resourceOffers(mock, Arrays.asList(offer));
+        scheduler.resourceOffers(mock, Collections.singletonList(offer));
 
         assertNull(mock.launchedTask);
 
@@ -122,7 +123,7 @@ public class IgniteSchedulerSelfTest extends TestCase {
 
         scheduler.setClusterProps(clustProp);
 
-        scheduler.resourceOffers(mock, Arrays.asList(offer));
+        scheduler.resourceOffers(mock, Collections.singletonList(offer));
 
         assertNotNull(mock.launchedTask);
         assertEquals(1, mock.launchedTask.size());
@@ -134,7 +135,7 @@ public class IgniteSchedulerSelfTest extends TestCase {
 
         mock.clear();
 
-        scheduler.resourceOffers(mock, Arrays.asList(offer));
+        scheduler.resourceOffers(mock, Collections.singletonList(offer));
 
         assertNull(mock.launchedTask);
 
@@ -160,7 +161,7 @@ public class IgniteSchedulerSelfTest extends TestCase {
         double totalMem = 0, totalCpu = 0;
 
         for (int i = 0; i < 2; i++) {
-            scheduler.resourceOffers(mock, Arrays.asList(offer));
+            scheduler.resourceOffers(mock, Collections.singletonList(offer));
 
             assertNotNull(mock.launchedTask);
             assertEquals(1, mock.launchedTask.size());
@@ -176,7 +177,7 @@ public class IgniteSchedulerSelfTest extends TestCase {
         assertEquals(2.0, totalCpu);
         assertEquals(2000.0, totalMem);
 
-        scheduler.resourceOffers(mock, Arrays.asList(offer));
+        scheduler.resourceOffers(mock, Collections.singletonList(offer));
 
         assertNull(mock.launchedTask);
 
@@ -198,7 +199,7 @@ public class IgniteSchedulerSelfTest extends TestCase {
 
         scheduler.setClusterProps(clustProp);
 
-        scheduler.resourceOffers(mock, Arrays.asList(offer));
+        scheduler.resourceOffers(mock, Collections.singletonList(offer));
 
         assertNotNull(mock.declinedOffer);
 
@@ -218,13 +219,39 @@ public class IgniteSchedulerSelfTest extends TestCase {
 
         scheduler.setClusterProps(clustProp);
 
-        scheduler.resourceOffers(mock, Arrays.asList(offer));
+        scheduler.resourceOffers(mock, Collections.singletonList(offer));
 
         assertNotNull(mock.declinedOffer);
 
         assertEquals(offer.getId(), mock.declinedOffer);
     }
 
+    /**
+     * @throws Exception If failed.
+     */
+    public void testHosthameConstraint() throws Exception {
+        Protos.Offer offer = createOffer("hostname", 8, 10240);
+
+        DriverMock mock = new DriverMock();
+
+        ClusterProperties clustProp = new ClusterProperties();
+        clustProp.hostnameConstraint(Pattern.compile("hostname"));
+
+        scheduler.setClusterProps(clustProp);
+
+        scheduler.resourceOffers(mock, Collections.singletonList(offer));
+
+        assertNotNull(mock.declinedOffer);
+
+        assertEquals(offer.getId(), mock.declinedOffer);
+
+        offer = createOffer("hostnameAccept", 8, 10240);
+
+        scheduler.resourceOffers(mock, Collections.singletonList(offer));
+
+        assertNotNull(mock.launchedTask);
+        assertEquals(1, mock.launchedTask.size());
+    }
 
     /**
      * @param resourceType Resource type.


[21/37] incubator-ignite git commit: Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-857

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-857


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

Branch: refs/heads/ignite-gg-10369
Commit: b7f554a65270c6eacb6dc7ff1c28866abdfd0a15
Parents: 8574a19 4be517c
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed May 27 19:17:02 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed May 27 19:17:02 2015 +0300

----------------------------------------------------------------------
 .gitignore                                      |   3 +-
 LICENSE                                         | 238 ++++++++
 LICENSE.txt                                     | 238 --------
 NOTICE                                          |  12 +
 NOTICE.txt                                      |  12 -
 assembly/release-base.xml                       |   4 +-
 dev-tools/build.gradle                          |  35 +-
 dev-tools/gradle/wrapper/gradle-wrapper.jar     | Bin 0 -> 51017 bytes
 .../gradle/wrapper/gradle-wrapper.properties    |   6 +
 dev-tools/gradlew                               | 164 ++++++
 dev-tools/slurp.sh                              |  76 +++
 dev-tools/src/main/groovy/jiraslurp.groovy      | 570 +++++++++++++++----
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 .../ClientAbstractConnectivitySelfTest.java     |  14 +
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 modules/core/src/main/java/META-INF/LICENSE     | 238 ++++++++
 modules/core/src/main/java/META-INF/NOTICE      |  12 +
 .../internal/GridEventConsumeHandler.java       |  26 +
 .../apache/ignite/internal/IgniteKernal.java    |  26 +-
 .../interop/InteropAwareEventFilter.java        |  37 ++
 .../internal/interop/InteropException.java      |  71 +++
 .../interop/InteropLocalEventListener.java      |  28 +
 .../interop/InteropNoCallbackException.java     |  50 ++
 .../managers/communication/GridIoManager.java   |   6 +-
 .../GridLifecycleAwareMessageFilter.java        |   5 +-
 .../eventstorage/GridEventStorageManager.java   |  24 +-
 .../processors/cache/GridCacheAdapter.java      |  16 +-
 .../processors/cache/GridCacheIoManager.java    |   6 +-
 .../GridCachePartitionExchangeManager.java      |   4 +-
 .../processors/cache/IgniteCacheProxy.java      |   3 +-
 .../distributed/GridDistributedTxMapping.java   |   5 +-
 .../GridDistributedTxRemoteAdapter.java         |  10 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   6 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   1 +
 .../dht/atomic/GridDhtAtomicCache.java          |   4 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |  42 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |  36 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   4 +-
 .../distributed/near/GridNearCacheEntry.java    |   2 +-
 .../cache/distributed/near/GridNearTxLocal.java |   5 +-
 .../near/GridNearTxPrepareFutureAdapter.java    |   9 +-
 .../near/GridNearTxPrepareResponse.java         |  28 +-
 .../transactions/IgniteTxLocalAdapter.java      |   4 +-
 .../cache/transactions/IgniteTxManager.java     |  24 +
 .../processors/query/GridQueryIndexing.java     |  16 +
 .../processors/query/GridQueryProcessor.java    |  52 +-
 .../ignite/internal/util/IgniteUtils.java       |   6 +-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../cache/IgniteCacheNearLockValueSelfTest.java | 145 +++++
 .../RandomEvictionPolicyCacheSizeSelfTest.java  |  72 +++
 .../IgniteCacheEvictionSelfTestSuite.java       |   1 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 modules/extdata/p2p/pom.xml                     |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 .../query/h2/GridH2IndexingGeoSelfTest.java     |  20 +-
 modules/hadoop/pom.xml                          |   2 +-
 .../processors/hadoop/v2/HadoopV2Context.java   |  10 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  17 +
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |  18 +-
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |   4 +-
 .../query/h2/sql/GridSqlOperationType.java      |   2 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |   2 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   2 +-
 .../local/IgniteCacheLocalQuerySelfTest.java    |   6 +
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 .../visor/commands/node/VisorNodeCommand.scala  |   2 +-
 .../commands/tasks/VisorTasksCommand.scala      |   2 +-
 .../scala/org/apache/ignite/visor/visor.scala   |  63 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 .../config/benchmark-client-mode.properties     |  89 +++
 modules/yardstick/pom.xml                       |   2 +-
 parent/pom.xml                                  |   5 +-
 pom.xml                                         |  56 +-
 96 files changed, 2238 insertions(+), 522 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b7f554a6/pom.xml
----------------------------------------------------------------------


[37/37] incubator-ignite git commit: #ignite-gg-10369: small fixes.

Posted by sb...@apache.org.
#ignite-gg-10369: small fixes.


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

Branch: refs/heads/ignite-gg-10369
Commit: a7301274d26b42a791a8c343974179d63bcdb49e
Parents: ee83aae
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri May 29 00:01:46 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri May 29 00:01:46 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/GridJavaProcess.java   | 24 +++++++++++---------
 1 file changed, 13 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a7301274/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
index 82f4064..6baa179 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
@@ -128,25 +128,27 @@ public final class GridJavaProcess {
         gjProc.log = log;
         gjProc.procKilledC = procKilledC;
 
-        String javaBin = System.getProperty("java.home") + File.separator + "bin" + File.separator + "java";
-        String classpath = System.getProperty("java.class.path");
-        String sfcp = System.getProperty("surefire.test.class.path");
-
-        if (sfcp != null)
-            classpath += System.getProperty("path.separator") + sfcp;
-
-        if (cp != null)
-            classpath += System.getProperty("path.separator") + cp;
-
         List<String> procParams = params == null || params.isEmpty() ?
             Collections.<String>emptyList() : Arrays.asList(params.split(" "));
 
         List<String> procCommands = new ArrayList<>();
 
+        String javaBin = System.getProperty("java.home") + File.separator + "bin" + File.separator + "java";
+
         procCommands.add(javaBin);
         procCommands.addAll(jvmArgs == null ? U.jvmArgs() : jvmArgs);
 
-        if (!jvmArgs.contains("-cp")) {
+        if (!jvmArgs.contains("-cp") || !jvmArgs.contains("-classpath")) {
+            String classpath = System.getProperty("java.class.path");
+
+            String sfcp = System.getProperty("surefire.test.class.path");
+
+            if (sfcp != null)
+                classpath += System.getProperty("path.separator") + sfcp;
+
+            if (cp != null)
+                classpath += System.getProperty("path.separator") + cp;
+
             procCommands.add("-cp");
             procCommands.add(classpath);
         }


[36/37] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-gg-10369

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-gg-10369


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

Branch: refs/heads/ignite-gg-10369
Commit: ee83aae6e4bdfdb488a9f528ecf7584de7f0bd0e
Parents: f23566e 56bda96
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu May 28 23:28:31 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu May 28 23:28:31 2015 +0300

----------------------------------------------------------------------
 dev-tools/src/main/groovy/jiraslurp.groovy      |   3 +
 .../shmem/IpcSharedMemoryServerEndpoint.java    |   2 +-
 modules/mesos/README.txt                        |  28 +
 modules/mesos/licenses/apache-2.0.txt           | 202 ++++++++
 modules/mesos/licenses/jetty-epl-license.txt    |  69 +++
 modules/mesos/pom.xml                           |  95 ++++
 .../apache/ignite/mesos/ClusterProperties.java  | 519 +++++++++++++++++++
 .../apache/ignite/mesos/IgniteFramework.java    | 119 +++++
 .../apache/ignite/mesos/IgniteScheduler.java    | 361 +++++++++++++
 .../org/apache/ignite/mesos/IgniteTask.java     |  86 +++
 .../org/apache/ignite/mesos/package-info.java   |  22 +
 .../ignite/mesos/resource/IgniteProvider.java   | 234 +++++++++
 .../ignite/mesos/resource/JettyServer.java      |  61 +++
 .../ignite/mesos/resource/ResourceHandler.java  | 142 +++++
 .../ignite/mesos/resource/ResourceProvider.java | 120 +++++
 .../ignite/mesos/resource/package-info.java     |  22 +
 .../main/resources/ignite-default-config.xml    |  35 ++
 .../org/apache/ignite/IgniteMesosTestSuite.java |  38 ++
 .../ignite/mesos/IgniteSchedulerSelfTest.java   | 464 +++++++++++++++++
 pom.xml                                         |   1 +
 scripts/git-patch-prop.sh                       |   2 +-
 21 files changed, 2623 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[03/37] incubator-ignite git commit: #ignite-857 Added AmazonScheduler.

Posted by sb...@apache.org.
#ignite-857 Added AmazonScheduler.


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

Branch: refs/heads/ignite-gg-10369
Commit: fbdcb70677249ed6aceab5057756e7eb04c89c04
Parents: b896ec8
Author: nikolay tikhonov <nt...@gridgain.com>
Authored: Thu May 14 19:48:05 2015 +0300
Committer: nikolay tikhonov <nt...@gridgain.com>
Committed: Thu May 14 19:48:05 2015 +0300

----------------------------------------------------------------------
 .../ignite/messo/IgniteAmazonScheduler.java     | 81 ++++++++++++++++++++
 .../apache/ignite/messo/IgniteFramework.java    | 27 ++++++-
 .../apache/ignite/messo/IgniteScheduler.java    | 67 ++++++++++------
 3 files changed, 149 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fbdcb706/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteAmazonScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteAmazonScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteAmazonScheduler.java
new file mode 100644
index 0000000..250fec6
--- /dev/null
+++ b/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteAmazonScheduler.java
@@ -0,0 +1,81 @@
+/*
+ * 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.messo;
+
+import org.apache.mesos.*;
+
+/**
+ * TODO
+ */
+public class IgniteAmazonScheduler extends IgniteScheduler {
+    /** */
+    public static final String AMAZON = "amazon";
+
+    /** Amazon credential. */
+    private final String accessKey, secretKey;
+
+    /**
+     * Constructor.
+     *
+     * @param accessKey Access key.
+     * @param secretKey Secret key.
+     */
+    public IgniteAmazonScheduler(String accessKey, String secretKey) {
+        assert accessKey != null;
+        assert secretKey != null;
+
+        this.accessKey = accessKey;
+        this.secretKey = secretKey;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Protos.TaskInfo createTask(Protos.Offer offer, Pair<Double, Double> cpuMem,
+        Protos.TaskID taskId) {
+        // Docker image info.
+        Protos.ContainerInfo.DockerInfo.Builder docker = Protos.ContainerInfo.DockerInfo.newBuilder()
+            .setImage(IMAGE)
+            .setNetwork(Protos.ContainerInfo.DockerInfo.Network.HOST);
+
+        // Container info.
+        Protos.ContainerInfo.Builder cont = Protos.ContainerInfo.newBuilder();
+        cont.setType(Protos.ContainerInfo.Type.DOCKER);
+        cont.setDocker(docker.build());
+
+        return Protos.TaskInfo.newBuilder()
+            .setName("task " + taskId.getValue())
+            .setTaskId(taskId)
+            .setSlaveId(offer.getSlaveId())
+            .addResources(Protos.Resource.newBuilder()
+                .setName(CPUS)
+                .setType(Protos.Value.Type.SCALAR)
+                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem._1)))
+            .addResources(Protos.Resource.newBuilder()
+                .setName(MEM)
+                .setType(Protos.Value.Type.SCALAR)
+                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem._2)))
+            .setContainer(cont)
+            .setCommand(Protos.CommandInfo.newBuilder()
+                .setShell(false)
+                .addArguments(STARTUP_SCRIPT)
+                .addArguments(String.valueOf(cpuMem._2.intValue()))
+                .addArguments(AMAZON)
+                .addArguments(accessKey)
+                .addArguments(secretKey))
+            .build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fbdcb706/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteFramework.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteFramework.java b/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteFramework.java
index 54d34c1..dfc3eb2 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteFramework.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteFramework.java
@@ -28,6 +28,8 @@ public class IgniteFramework {
      * @param args Args
      */
     public static void main(String[] args) {
+        checkArgs(args);
+
         final int frameworkFailoverTimeout = 0;
 
         Protos.FrameworkInfo.Builder frameworkBuilder = Protos.FrameworkInfo.newBuilder()
@@ -41,7 +43,7 @@ public class IgniteFramework {
         }
 
         // create the scheduler
-        final Scheduler scheduler = new IgniteScheduler();
+        final Scheduler scheduler = createIgniteScheduler(args);
 
         // create the driver
         MesosSchedulerDriver driver;
@@ -80,4 +82,27 @@ public class IgniteFramework {
 
         System.exit(status);
     }
+
+    /**
+     * @param args Arguments.
+     * @return Ignite scheduler.
+     */
+    private static IgniteScheduler createIgniteScheduler(String args[]) {
+        if (args.length >= 3 && args[1].equals(IgniteAmazonScheduler.AMAZON))
+            return new IgniteAmazonScheduler(args[2], args[3]);
+        else
+            return new IgniteScheduler();
+    }
+
+    /**
+     * Check input arguments.
+     *
+     * @param args Arguments.
+     */
+    private static void checkArgs(String[] args) {
+        if (args.length == 0)
+            throw new IllegalArgumentException("Illegal arguments.");
+
+        // TODO: add more
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fbdcb706/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java
index 3cb63e6..a029e69 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java
@@ -30,6 +30,9 @@ public class IgniteScheduler implements Scheduler {
     /** Docker image name. */
     public static final String IMAGE = "apacheignite/ignite-docker";
 
+    /** Startup sctipt path. */
+    public static final String STARTUP_SCRIPT = "/home/ignite/startup.sh";
+
     /** Cpus. */
     public static final String CPUS = "cpus";
 
@@ -76,32 +79,8 @@ public class IgniteScheduler implements Scheduler {
 
             log.info("Launching task {}", taskId.getValue());
 
-            // Docker image info.
-            Protos.ContainerInfo.DockerInfo.Builder docker = Protos.ContainerInfo.DockerInfo.newBuilder()
-                .setImage(IMAGE)
-                .setNetwork(Protos.ContainerInfo.DockerInfo.Network.HOST);
-
-            // Container info.
-            Protos.ContainerInfo.Builder cont = Protos.ContainerInfo.newBuilder();
-            cont.setType(Protos.ContainerInfo.Type.DOCKER);
-            cont.setDocker(docker.build());
-
             // Create task to run.
-            Protos.TaskInfo task = Protos.TaskInfo.newBuilder()
-                .setName("task " + taskId.getValue())
-                .setTaskId(taskId)
-                .setSlaveId(offer.getSlaveId())
-                .addResources(Protos.Resource.newBuilder()
-                    .setName(CPUS)
-                    .setType(Protos.Value.Type.SCALAR)
-                    .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem._1)))
-                .addResources(Protos.Resource.newBuilder()
-                    .setName(MEM)
-                    .setType(Protos.Value.Type.SCALAR)
-                    .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem._2)))
-                .setContainer(cont)
-                .setCommand(Protos.CommandInfo.newBuilder().setShell(false))
-                .build();
+            Protos.TaskInfo task = createTask(offer, cpuMem, taskId);
 
             schedulerDriver.launchTasks(Collections.singletonList(offer.getId()),
                 Collections.singletonList(task),
@@ -110,6 +89,44 @@ public class IgniteScheduler implements Scheduler {
     }
 
     /**
+     * Create Task.
+     * @param offer Offer.
+     * @param cpuMem Cpu and mem on slave.
+     * @param taskId Task id.
+     * @return Task.
+     */
+    protected Protos.TaskInfo createTask(Protos.Offer offer, Pair<Double, Double> cpuMem, Protos.TaskID taskId) {
+        // Docker image info.
+        Protos.ContainerInfo.DockerInfo.Builder docker = Protos.ContainerInfo.DockerInfo.newBuilder()
+                .setImage(IMAGE)
+                .setNetwork(Protos.ContainerInfo.DockerInfo.Network.HOST);
+
+        // Container info.
+        Protos.ContainerInfo.Builder cont = Protos.ContainerInfo.newBuilder();
+        cont.setType(Protos.ContainerInfo.Type.DOCKER);
+        cont.setDocker(docker.build());
+
+        return Protos.TaskInfo.newBuilder()
+            .setName("task " + taskId.getValue())
+            .setTaskId(taskId)
+            .setSlaveId(offer.getSlaveId())
+            .addResources(Protos.Resource.newBuilder()
+                .setName(CPUS)
+                .setType(Protos.Value.Type.SCALAR)
+                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem._1)))
+            .addResources(Protos.Resource.newBuilder()
+                .setName(MEM)
+                .setType(Protos.Value.Type.SCALAR)
+                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem._2)))
+            .setContainer(cont)
+            .setCommand(Protos.CommandInfo.newBuilder()
+                .setShell(false)
+                .addArguments(STARTUP_SCRIPT)
+                .addArguments(String.valueOf(cpuMem._2.intValue())))
+            .build();
+    }
+
+    /**
      * Check slave resources and return resources infos.
      *
      * @param offer Offer request.


[35/37] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-sprint-5


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

Branch: refs/heads/ignite-gg-10369
Commit: 56bda960f47fc0b929c8cd5b2bfcb5a114140408
Parents: bac7f79 43f7aaf
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu May 28 19:21:06 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu May 28 19:21:06 2015 +0300

----------------------------------------------------------------------
 dev-tools/src/main/groovy/jiraslurp.groovy                        | 3 +++
 .../internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java    | 2 +-
 scripts/git-patch-prop.sh                                         | 2 +-
 3 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[02/37] incubator-ignite git commit: #IGNITE-857 Decline offer with doesn't match by mem and cpu

Posted by sb...@apache.org.
#IGNITE-857 Decline offer with doesn't match by mem and cpu


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

Branch: refs/heads/ignite-gg-10369
Commit: b896ec8b311b65af97a53d8271ee4bfe69e1ebd3
Parents: 934bf69
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Fri May 8 16:42:48 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Fri May 8 16:42:48 2015 +0300

----------------------------------------------------------------------
 modules/mesos/pom.xml                              |  2 +-
 .../org/apache/ignite/messo/IgniteScheduler.java   | 17 ++++++++---------
 2 files changed, 9 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b896ec8b/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index 8c50085..4d19b11 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -59,7 +59,7 @@
                 <version>2.4.1</version>
                 <configuration>
                     <descriptorRefs>
-                        <descriptorRef>fat-jar</descriptorRef>
+                        <descriptorRef>jar-with-dependencies</descriptorRef>
                     </descriptorRefs>
                 </configuration>
                 <executions>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b896ec8b/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java
index bddcec6..3cb63e6 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/messo/IgniteScheduler.java
@@ -60,15 +60,15 @@ public class IgniteScheduler implements Scheduler {
     @Override public void resourceOffers(SchedulerDriver schedulerDriver, List<Protos.Offer> offers) {
         log.info("resourceOffers() with {} offers", offers.size());
 
-        List<Protos.OfferID> offerIDs = new ArrayList<>(offers.size());
-        List<Protos.TaskInfo> tasks = new ArrayList<>(offers.size());
-
         for (Protos.Offer offer : offers) {
             Pair<Double, Double> cpuMem = checkOffer(offer);
 
-            //
-            if (cpuMem == null)
+            // Decline offer which doesn't match by mem or cpu.
+            if (cpuMem == null) {
+                schedulerDriver.declineOffer(offer.getId());
+
                 continue;
+            }
 
             // Generate a unique task ID.
             Protos.TaskID taskId = Protos.TaskID.newBuilder()
@@ -103,11 +103,10 @@ public class IgniteScheduler implements Scheduler {
                 .setCommand(Protos.CommandInfo.newBuilder().setShell(false))
                 .build();
 
-            offerIDs.add(offer.getId());
-            tasks.add(task);
+            schedulerDriver.launchTasks(Collections.singletonList(offer.getId()),
+                Collections.singletonList(task),
+                Protos.Filters.newBuilder().setRefuseSeconds(1).build());
         }
-
-        schedulerDriver.launchTasks(offerIDs, tasks, Protos.Filters.newBuilder().setRefuseSeconds(1).build());
     }
 
     /**


[19/37] incubator-ignite git commit: #IGNITE-857 WIP

Posted by sb...@apache.org.
#IGNITE-857 WIP


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

Branch: refs/heads/ignite-gg-10369
Commit: 8deb57728022d79239602070bdd90857606d0942
Parents: 9a951e0
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed May 27 16:34:50 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed May 27 16:34:50 2015 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/mesos/IgniteScheduler.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8deb5772/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index 9b8553e..17daf45 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -87,7 +87,7 @@ public class IgniteScheduler implements Scheduler {
 
                 // Generate a unique task ID.
                 Protos.TaskID taskId = Protos.TaskID.newBuilder()
-                    .setValue(taskIdGenerator.incrementAndGet() + "-" + clusterProps.clusterName()).build();
+                    .setValue(Integer.toString(taskIdGenerator.incrementAndGet())).build();
 
                 log.info("Launching task: [{}]", igniteTask);
 


[09/37] incubator-ignite git commit: #IGNITE-857 Added web-server. Support set ignite version.

Posted by sb...@apache.org.
#IGNITE-857 Added web-server. Support set ignite version.


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

Branch: refs/heads/ignite-gg-10369
Commit: 4b482187871184e4e8d8086aae870ff47c057275
Parents: ae8bcf8
Author: nikolay tikhonov <nt...@gridgain.com>
Authored: Mon May 25 17:10:50 2015 +0300
Committer: nikolay tikhonov <nt...@gridgain.com>
Committed: Mon May 25 17:10:50 2015 +0300

----------------------------------------------------------------------
 modules/mesos/pom.xml                           |  22 ++
 .../apache/ignite/mesos/ClusterProperties.java  | 263 +++++++++++++++++++
 .../apache/ignite/mesos/ClusterResources.java   | 160 -----------
 .../apache/ignite/mesos/IgniteFramework.java    |  64 +++--
 .../apache/ignite/mesos/IgniteScheduler.java    |  67 +++--
 .../ignite/mesos/resource/IgniteProvider.java   | 234 +++++++++++++++++
 .../mesos/resource/ResourceController.java      | 130 +++++++++
 .../ignite/mesos/resource/ResourceProvider.java | 120 +++++++++
 .../main/resources/ignite-default-config.xml    |  35 +++
 modules/mesos/src/main/resources/log4j2.xml     |  35 +++
 10 files changed, 924 insertions(+), 206 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4b482187/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index 5ce3e5c..4aa0dae 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -27,6 +27,10 @@
     <artifactId>ignite-mesos</artifactId>
     <version>1.1.0-SNAPSHOT</version>
 
+    <properties>
+        <version.grizzly>2.16</version.grizzly>
+    </properties>
+
     <dependencies>
         <dependency>
             <groupId>org.apache.mesos</groupId>
@@ -41,6 +45,24 @@
         </dependency>
 
         <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-core</artifactId>
+            <version>2.0.2</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-slf4j-impl</artifactId>
+            <version>2.0.2</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.glassfish.jersey.containers</groupId>
+            <artifactId>jersey-container-grizzly2-http</artifactId>
+            <version>${version.grizzly}</version>
+        </dependency>
+
+        <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>
             <version>4.11</version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4b482187/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
new file mode 100644
index 0000000..bb7f7a4
--- /dev/null
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
@@ -0,0 +1,263 @@
+/*
+ * 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.mesos;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Cluster settings.
+ */
+public class ClusterProperties {
+    /** Unlimited. */
+    public static final double UNLIMITED = -1;
+
+    /** */
+    public static final String MESOS_MASTER_URL = "MESOS_MASTER_URL";
+
+    /** */
+    public static final String DEFAULT_MESOS_MASTER_URL = "zk://localhost:2181/mesos";
+
+    /** Mesos master url. */
+    private String mesosUrl = DEFAULT_MESOS_MASTER_URL;
+
+    /** */
+    public static final String IGNITE_RESOURCE_CPU_CORES = "IGNITE_RESOURCE_CPU_CORES";
+
+    /** CPU limit. */
+    private double cpu = UNLIMITED;
+
+    /** */
+    public static final String IGNITE_RESOURCE_MEM_MB = "IGNITE_RESOURCE_MEM_MB";
+
+    /** Memory limit. */
+    private double mem = UNLIMITED;
+
+    /** */
+    public static final String IGNITE_RESOURCE_DISK_MB = "IGNITE_RESOURCE_DISK_MB";
+
+    /** Disk space limit. */
+    private double disk = UNLIMITED;
+
+    /** */
+    public static final String IGNITE_RESOURCE_NODE_CNT = "IGNITE_RESOURCE_NODE_CNT";
+
+    /** Node count limit. */
+    private double nodeCnt = UNLIMITED;
+
+    /** */
+    public static final String IGNITE_RESOURCE_MIN_CPU_CNT_PER_NODE = "IGNITE_RESOURCE_MIN_CPU_CNT_PER_NODE";
+
+    /** */
+    public static final double DEFAULT_RESOURCE_MIN_CPU = 2;
+
+    /** Min memory per node. */
+    private double minCpu = DEFAULT_RESOURCE_MIN_CPU;
+
+    /** */
+    public static final String IGNITE_RESOURCE_MIN_MEMORY_PER_NODE = "IGNITE_RESOURCE_MIN_MEMORY_PER_NODE";
+
+    /** */
+    public static final double DEFAULT_RESOURCE_MIN_MEM = 256;
+
+    /** Min memory per node. */
+    private double minMemory = DEFAULT_RESOURCE_MIN_MEM;
+
+    /** */
+    public static final String IGNITE_VERSION = "IGNITE_VERSION";
+
+    /** */
+    public static final String DEFAULT_IGNITE_VERSION = "latest";
+
+    /** Ignite version. */
+    private String igniteVer = DEFAULT_IGNITE_VERSION;
+
+    /** */
+    public static final String IGNITE_WORK_DIR = "IGNITE_WORK_DIR";
+
+    /** */
+    public static final String DEFAULT_IGNITE_WORK_DIR = "~/ignite-releases";
+
+    /** Ignite version. */
+    private String igniteWorkDir = DEFAULT_IGNITE_WORK_DIR;
+
+    /** */
+    public static final String IGNITE_USERS_LIBS = "IGNITE_USERS_LIBS";
+
+    /** Path to users libs. */
+    private String userLibs = null;
+
+    /** */
+    public static final String IGNITE_CONFIG_XML = "IGNITE_XML_CONFIG";
+
+    /** Ignite config. */
+    private String igniteCfg = null;
+
+    /** */
+    public ClusterProperties() {
+        // No-op.
+    }
+
+    /**
+     * @return CPU count limit.
+     */
+    public double cpus(){
+        return cpu;
+    }
+
+    /**
+     * @return mem limit.
+     */
+    public double memory() {
+        return mem;
+    }
+
+    /**
+     * @return disk limit.
+     */
+    public double disk() {
+        return disk;
+    }
+
+    /**
+     * @return instance count limit.
+     */
+    public double instances() {
+        return nodeCnt;
+    }
+
+    /**
+     * @return min memory per node.
+     */
+    public double minMemoryPerNode() {
+        return minMemory;
+    }
+
+    /**
+     * @return min cpu count per node.
+     */
+    public double minCpuPerNode() {
+        return minCpu;
+    }
+
+    /**
+     * @return Ignite version.
+     */
+    public String igniteVer() {
+        return igniteVer;
+    }
+
+    /**
+     * @return Working directory.
+     */
+    public String igniteWorkDir() {
+        return igniteWorkDir;
+    }
+
+    /**
+     * @return User's libs.
+     */
+    public String userLibs() {
+        return userLibs;
+    }
+
+    /**
+     * @return Ignite configuration.
+     */
+    public String igniteCfg() {
+        return igniteCfg;
+    }
+
+    /**
+     * @return Master url.
+     */
+    public String masterUrl() {
+        return mesosUrl;
+    }
+
+    /**
+     * @param config path to config file.
+     * @return Cluster configuration.
+     */
+    public static ClusterProperties from(String config) {
+        try {
+            Properties props = null;
+
+            if (config != null) {
+                props = new Properties();
+
+                props.load(new FileInputStream(config));
+            }
+
+            ClusterProperties prop = new ClusterProperties();
+
+            prop.mesosUrl = getStringProperty(MESOS_MASTER_URL, props, DEFAULT_MESOS_MASTER_URL);
+
+            prop.cpu = getDoubleProperty(IGNITE_RESOURCE_CPU_CORES, props, UNLIMITED);
+            prop.mem = getDoubleProperty(IGNITE_RESOURCE_MEM_MB, props, UNLIMITED);
+            prop.disk = getDoubleProperty(IGNITE_RESOURCE_DISK_MB, props, UNLIMITED);
+            prop.nodeCnt = getDoubleProperty(IGNITE_RESOURCE_NODE_CNT, props, UNLIMITED);
+            prop.minCpu = getDoubleProperty(IGNITE_RESOURCE_MIN_CPU_CNT_PER_NODE, props, DEFAULT_RESOURCE_MIN_CPU);
+            prop.minMemory = getDoubleProperty(IGNITE_RESOURCE_MIN_MEMORY_PER_NODE, props, DEFAULT_RESOURCE_MIN_MEM);
+
+            prop.igniteVer = getStringProperty(IGNITE_VERSION, props, DEFAULT_IGNITE_VERSION);
+            prop.igniteWorkDir = getStringProperty(IGNITE_WORK_DIR, props, DEFAULT_IGNITE_WORK_DIR);
+            prop.igniteCfg = getStringProperty(IGNITE_CONFIG_XML, props, null);
+            prop.userLibs = getStringProperty(IGNITE_USERS_LIBS, props, null);
+
+            return prop;
+        }
+        catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * @param name Property name.
+     * @param fileProps Property file.
+     * @return Property value.
+     */
+    private static double getDoubleProperty(String name, Properties fileProps, Double defaultVal) {
+        if (fileProps != null && fileProps.containsKey(name))
+            return Double.valueOf(fileProps.getProperty(name));
+
+        String property = System.getProperty(name);
+
+        if (property == null)
+            property = System.getenv(name);
+
+        return property == null ? defaultVal : Double.valueOf(property);
+    }
+
+    /**
+     * @param name Property name.
+     * @param fileProps Property file.
+     * @return Property value.
+     */
+    private static String getStringProperty(String name, Properties fileProps, String defaultVal) {
+        if (fileProps != null && fileProps.containsKey(name))
+            return fileProps.getProperty(name);
+
+        String property = System.getProperty(name);
+
+        if (property == null)
+            property = System.getenv(name);
+
+        return property == null ? defaultVal : property;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4b482187/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterResources.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterResources.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterResources.java
deleted file mode 100644
index 1887530..0000000
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterResources.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
- * 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.mesos;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Cluster settings.
- */
-public class ClusterResources {
-    /** Unlimited. */
-    public static final int DEFAULT_VALUE = -1;
-
-    /** */
-    public static final String IGNITE_RESOURCE_CPU_CORES = "IGNITE_RESOURCE_CPU_CORES";
-
-    /** CPU limit. */
-    private double cpu = DEFAULT_VALUE;
-
-    /** */
-    public static final String IGNITE_RESOURCE_MEM_MB = "IGNITE_RESOURCE_MEM_MB";
-
-    /** Memory limit. */
-    private double mem = DEFAULT_VALUE;
-
-    /** */
-    public static final String IGNITE_RESOURCE_DISK_MB = "IGNITE_RESOURCE_DISK_MB";
-
-    /** Disk space limit. */
-    private double disk = DEFAULT_VALUE;
-
-    /** */
-    public static final String IGNITE_RESOURCE_NODE_CNT = "IGNITE_RESOURCE_NODE_CNT";
-
-    /** Node count limit. */
-    private double nodeCnt = DEFAULT_VALUE;
-
-    /** */
-    public static final String IGNITE_RESOURCE_MIN_CPU_CNT_PER_NODE = "IGNITE_RESOURCE_MIN_CPU_CNT_PER_NODE";
-
-    /** Min memory per node. */
-    private int minCpu = 2;
-
-    /** */
-    public static final String IGNITE_RESOURCE_MIN_MEMORY_PER_NODE = "IGNITE_RESOURCE_MIN_MEMORY_PER_NODE";
-
-    /** Min memory per node. */
-    private int minMemoryCnt = 256;
-
-    /** */
-    public ClusterResources() {
-        // No-op.
-    }
-
-    /**
-     * @return CPU count limit.
-     */
-    public double cpus(){
-        return cpu;
-    }
-
-    /**
-     * @return mem limit.
-     */
-    public double memory() {
-        return mem;
-    }
-
-    /**
-     * @return disk limit.
-     */
-    public double disk() {
-        return disk;
-    }
-
-    /**
-     * @return instance count limit.
-     */
-    public double instances() {
-        return nodeCnt;
-    }
-
-    /**
-     * @return min memory per node.
-     */
-    public int minMemoryPerNode() {
-        return minMemoryCnt;
-    }
-
-    /**
-     * @return min cpu count per node.
-     */
-    public int minCpuPerNode() {
-        return minCpu;
-    }
-
-    /**
-     * @param config path to config file.
-     * @return Cluster configuration.
-     */
-    public static ClusterResources from(String config) {
-        try {
-            Properties props = null;
-
-            if (config != null) {
-                props = new Properties();
-
-                props.load(new FileInputStream(config));
-            }
-
-            ClusterResources resources = new ClusterResources();
-
-            resources.cpu = getProperty(IGNITE_RESOURCE_CPU_CORES, props);
-            resources.mem = getProperty(IGNITE_RESOURCE_MEM_MB, props);
-            resources.disk = getProperty(IGNITE_RESOURCE_DISK_MB, props);
-            resources.nodeCnt = getProperty(IGNITE_RESOURCE_NODE_CNT, props);
-
-            return resources;
-        }
-        catch (IOException e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     * @param name Property name.
-     * @param fileProps Property file.
-     * @return Property value.
-     */
-    private static double getProperty(String name, Properties fileProps) {
-        if (fileProps != null && fileProps.containsKey(name))
-            return Double.valueOf(fileProps.getProperty(name));
-
-        String property = System.getProperty(name);
-
-        if (property == null)
-            property = System.getenv(name);
-
-        if (property == null)
-            return DEFAULT_VALUE;
-
-        return Double.valueOf(property);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4b482187/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
index 2d74f71..0ff945b 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
@@ -18,22 +18,29 @@
 package org.apache.ignite.mesos;
 
 import com.google.protobuf.*;
+import org.apache.ignite.mesos.resource.*;
 import org.apache.mesos.*;
+import org.glassfish.grizzly.http.server.*;
+import org.glassfish.jersey.grizzly2.httpserver.*;
+import org.glassfish.jersey.server.*;
+
+import java.net.*;
 
 /**
  * TODO
  */
 public class IgniteFramework {
+
+    public static final String IGNITE_FRAMEWORK_NAME = "IgniteFramework";
+
     /**
-     * @param args Args [host:port] [resource limit]
+     * @param args Args
      */
-    public static void main(String[] args) {
-        checkArgs(args);
-
+    public static void main(String[] args) throws Exception {
         final int frameworkFailoverTimeout = 0;
 
         Protos.FrameworkInfo.Builder frameworkBuilder = Protos.FrameworkInfo.newBuilder()
-            .setName("IgniteFramework")
+            .setName(IGNITE_FRAMEWORK_NAME)
             .setUser("") // Have Mesos fill in the current user.
             .setFailoverTimeout(frameworkFailoverTimeout); // timeout in seconds
 
@@ -42,8 +49,26 @@ public class IgniteFramework {
             frameworkBuilder.setCheckpoint(true);
         }
 
+        ClusterProperties clusterProperties = ClusterProperties.from(args.length == 1 ? args[0] : null);
+
+        String baseUrl = String.format("http://%s:%d", formatInetAddress(InetAddress.getLocalHost()), 4444);
+
+        URI httpServerBaseUri = URI.create(baseUrl);
+
+        ResourceConfig rc = new ResourceConfig()
+            .registerInstances(new ResourceController(clusterProperties.userLibs(), clusterProperties.igniteCfg(),
+                clusterProperties.igniteWorkDir()));
+
+        HttpServer httpServer = GrizzlyHttpServerFactory.createHttpServer(httpServerBaseUri, rc);
+
+        ResourceProvider provider = new ResourceProvider();
+
+        IgniteProvider igniteProvider = new IgniteProvider(clusterProperties.igniteWorkDir());
+
+        provider.init(clusterProperties, igniteProvider, baseUrl);
+
         // Create the scheduler.
-        final Scheduler scheduler = new IgniteScheduler(ClusterResources.from(null));
+        Scheduler scheduler = new IgniteScheduler(clusterProperties, provider);
 
         // create the driver
         MesosSchedulerDriver driver;
@@ -67,31 +92,34 @@ public class IgniteFramework {
 
             frameworkBuilder.setPrincipal(System.getenv("DEFAULT_PRINCIPAL"));
 
-            driver = new MesosSchedulerDriver(scheduler, frameworkBuilder.build(), args[0], credential);
+            driver = new MesosSchedulerDriver(scheduler, frameworkBuilder.build(), clusterProperties.masterUrl(),
+                credential);
         }
         else {
             frameworkBuilder.setPrincipal("ignite-framework-java");
 
-            driver = new MesosSchedulerDriver(scheduler, frameworkBuilder.build(), args[0]);
+            driver = new MesosSchedulerDriver(scheduler, frameworkBuilder.build(), clusterProperties.masterUrl());
         }
 
         int status = driver.run() == Protos.Status.DRIVER_STOPPED ? 0 : 1;
 
+        httpServer.shutdown();
+
         // Ensure that the driver process terminates.
         driver.stop();
 
         System.exit(status);
     }
 
-    /**
-     * Check input arguments.
-     *
-     * @param args Arguments.
-     */
-    private static void checkArgs(String[] args) {
-        if (args.length == 0)
-            throw new IllegalArgumentException("Illegal arguments.");
-
-        // TODO: add more
+    public static String formatInetAddress(final InetAddress inetAddress) {
+        if (inetAddress instanceof Inet4Address) {
+            return inetAddress.getHostAddress();
+        }
+        else if (inetAddress instanceof Inet6Address) {
+            return String.format("[%s]", inetAddress.getHostAddress());
+        }
+        else
+            throw new IllegalArgumentException("InetAddress type: " + inetAddress.getClass().getName() +
+                " is not supported");
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4b482187/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index 9d10860..b1ff930 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.mesos;
 
+import org.apache.ignite.mesos.resource.*;
 import org.apache.mesos.*;
 import org.slf4j.*;
 
@@ -27,12 +28,6 @@ import java.util.concurrent.atomic.*;
  * TODO
  */
 public class IgniteScheduler implements Scheduler {
-    /** Docker image name. */
-    public static final String IMAGE = "apacheignite/ignite-docker";
-
-    /** Startup sctipt path. */
-    public static final String STARTUP_SCRIPT = "/home/ignite/startup.sh";
-
     /** Cpus. */
     public static final String CPUS = "cpus";
 
@@ -61,13 +56,18 @@ public class IgniteScheduler implements Scheduler {
     private Map<String, IgniteTask> tasks = new HashMap<>();
 
     /** Cluster resources. */
-    private ClusterResources clusterLimit;
+    private ClusterProperties clusterLimit;
+
+    /** Resource provider. */
+    private ResourceProvider resourceProvider;
 
     /**
-     * @param clusterLimit Resources limit.
+     * @param clusterLimit Cluster limit.
+     * @param resourceProvider Resource provider.
      */
-    public IgniteScheduler(ClusterResources clusterLimit) {
+    public IgniteScheduler(ClusterProperties clusterLimit, ResourceProvider resourceProvider) {
         this.clusterLimit = clusterLimit;
+        this.resourceProvider = resourceProvider;
     }
 
     /** {@inheritDoc} */
@@ -114,6 +114,7 @@ public class IgniteScheduler implements Scheduler {
         }
     }
 
+
     /**
      * Create Task.
      *
@@ -123,20 +124,36 @@ public class IgniteScheduler implements Scheduler {
      * @return Task.
      */
     protected Protos.TaskInfo createTask(Protos.Offer offer, IgniteTask igniteTask, Protos.TaskID taskId) {
-        // Docker image info.
-        Protos.ContainerInfo.DockerInfo.Builder docker = Protos.ContainerInfo.DockerInfo.newBuilder()
-            .setImage(IMAGE)
-            .setNetwork(Protos.ContainerInfo.DockerInfo.Network.HOST);
-
-        // Container info.
-        Protos.ContainerInfo.Builder cont = Protos.ContainerInfo.newBuilder();
-        cont.setType(Protos.ContainerInfo.Type.DOCKER);
-        cont.setDocker(docker.build());
+        Protos.CommandInfo.Builder builder = Protos.CommandInfo.newBuilder()
+            .setEnvironment(Protos.Environment.newBuilder().addVariables(Protos.Environment.Variable.newBuilder()
+                .setName("IGNITE_TCP_DISCOVERY_ADDRESSES")
+                .setValue(getAddress())))
+            .addUris(Protos.CommandInfo.URI.newBuilder()
+                .setValue(resourceProvider.igniteUrl())
+                .setExtract(true))
+            .addUris(Protos.CommandInfo.URI.newBuilder().setValue(resourceProvider.igniteConfigUrl()));
+
+        if (resourceProvider.resourceUrl() != null) {
+            for (String url : resourceProvider.resourceUrl())
+                builder.addUris(Protos.CommandInfo.URI.newBuilder().setValue(url));
+
+            builder.setValue("cp *.jar ./gridgain-community-*/libs/ "
+                + "&& ./gridgain-community-*/bin/ignite.sh "
+                + resourceProvider.configName()
+                + " -J-Xmx" + String.valueOf((int) igniteTask.mem() + "m")
+                + " -J-Xms" + String.valueOf((int) igniteTask.mem()) + "m");
+        }
+        else
+            builder.setValue("./gridgain-community-*/bin/ignite.sh "
+                + resourceProvider.configName()
+                + " -J-Xmx" + String.valueOf((int) igniteTask.mem() + "m")
+                + " -J-Xms" + String.valueOf((int) igniteTask.mem()) + "m");
 
         return Protos.TaskInfo.newBuilder()
             .setName("Ignite node " + taskId.getValue())
             .setTaskId(taskId)
             .setSlaveId(offer.getSlaveId())
+            .setCommand(builder)
             .addResources(Protos.Resource.newBuilder()
                 .setName(CPUS)
                 .setType(Protos.Value.Type.SCALAR)
@@ -145,12 +162,6 @@ public class IgniteScheduler implements Scheduler {
                 .setName(MEM)
                 .setType(Protos.Value.Type.SCALAR)
                 .setScalar(Protos.Value.Scalar.newBuilder().setValue(igniteTask.mem())))
-            .setContainer(cont)
-            .setCommand(Protos.CommandInfo.newBuilder()
-                .setShell(false)
-                .addArguments(STARTUP_SCRIPT)
-                .addArguments(String.valueOf((int) igniteTask.mem()))
-                .addArguments(getAddress()))
             .build();
     }
 
@@ -227,11 +238,11 @@ public class IgniteScheduler implements Scheduler {
             totalDisk += task.disk();
         }
 
-        cpus = clusterLimit.cpus() == ClusterResources.DEFAULT_VALUE ? cpus :
+        cpus = clusterLimit.cpus() == ClusterProperties.UNLIMITED ? cpus :
             Math.min(clusterLimit.cpus() - totalCpus, cpus);
-        mem = clusterLimit.memory() == ClusterResources.DEFAULT_VALUE ? mem :
+        mem = clusterLimit.memory() == ClusterProperties.UNLIMITED ? mem :
             Math.min(clusterLimit.memory() - totalMem, mem);
-        disk = clusterLimit.disk() == ClusterResources.DEFAULT_VALUE ? disk :
+        disk = clusterLimit.disk() == ClusterProperties.UNLIMITED ? disk :
             Math.min(clusterLimit.disk() - totalDisk, disk);
 
         if (cpus > 0 && mem > 0)
@@ -253,7 +264,7 @@ public class IgniteScheduler implements Scheduler {
      * @return {@code True} if limit isn't violated else {@code false}.
      */
     private boolean checkLimit(double limit, double value) {
-        return limit == ClusterResources.DEFAULT_VALUE || limit <= value;
+        return limit == ClusterProperties.UNLIMITED || limit <= value;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4b482187/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/IgniteProvider.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/IgniteProvider.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/IgniteProvider.java
new file mode 100644
index 0000000..18ceb00
--- /dev/null
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/IgniteProvider.java
@@ -0,0 +1,234 @@
+/*
+ * 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.mesos.resource;
+
+import java.io.*;
+import java.net.*;
+import java.nio.channels.*;
+import java.util.*;
+
+/**
+ * TODO
+ */
+public class IgniteProvider {
+    /** */
+    public static final String DOWNLOAD_LINK = "http://tiny.cc/updater/download_community.php";
+
+    /** */
+    public static final String DIRECT_DOWNLOAD_LINK = "http://www.gridgain.com/media/gridgain-community-fabric-";
+
+    /** */
+    private String downloadFolder;
+
+    /** */
+    private String latestVersion = null;
+
+    /**
+     * @param downloadFolder Folder with ignite.
+     */
+    public IgniteProvider(String downloadFolder) {
+        this.downloadFolder = downloadFolder;
+    }
+
+    /**
+     * @return Latest ignite version.
+     */
+    public String getIgnite() {
+        File folder = checkDownloadFolder();
+
+        if (latestVersion == null) {
+            List<String> files = findIgnites(folder);
+
+            if (!files.isEmpty()) {
+                if (files.size() == 1)
+                    latestVersion = parseVersion(files.get(0));
+                else
+                    latestVersion = parseVersion(Collections.max(files, new Comparator<String>() {
+                        @Override public int compare(String f1, String f2) {
+                            if (f1.equals(f2))
+                                return 0;
+
+                            String[] ver1 = parseVersion(f1).split("\\.");
+                            String[] ver2 = parseVersion(f2).split("\\.");
+
+                            if (Integer.valueOf(ver1[0]) > Integer.valueOf(ver2[0])
+                                && Integer.valueOf(ver1[1]) > Integer.valueOf(ver2[1])
+                                && Integer.valueOf(ver1[2]) > Integer.valueOf(ver2[2]))
+
+                                return 1;
+                            else
+                                return -1;
+                        }
+                    }));
+            }
+        }
+
+        latestVersion = updateIgnite(latestVersion);
+
+        return "gridgain-community-fabric-" + latestVersion + ".zip";
+    }
+
+    /**
+     * @param folder Folder.
+     * @return Ignite archives.
+     */
+    private List<String> findIgnites(File folder) {
+        String[] files = folder.list();
+
+        List<String> ignites = new ArrayList<>();
+
+        if (files != null) {
+            for (String fileName : files) {
+                if (fileName.contains("gridgain-community-fabric-") && fileName.endsWith(".zip"))
+                    ignites.add(fileName);
+            }
+        }
+
+        return ignites;
+    }
+
+    /**
+     * @param version Ignite version.
+     * @return Ignite.
+     */
+    public String getIgnite(String version) {
+        File folder = checkDownloadFolder();
+
+        String[] ignites = folder.list();
+
+        String ignite = null;
+
+        if (ignites != null) {
+            for (String fileName : ignites) {
+                if (fileName.equals("gridgain-community-fabric-" + version + ".zip"))
+                    ignite = fileName;
+            }
+        }
+
+        if (ignite != null)
+            return ignite;
+
+        return downloadIgnite(version);
+    }
+
+    /**
+     * @param currentVersion The current latest version.
+     * @return Current version if the current version is latest; new ignite version otherwise.
+     */
+    private String updateIgnite(String currentVersion) {
+        try {
+            URL url;
+
+            if (currentVersion == null)
+                url = new URL(DOWNLOAD_LINK);
+            else
+                url = new URL(DOWNLOAD_LINK + "?version=" + currentVersion);
+
+            HttpURLConnection conn = (HttpURLConnection)url.openConnection();
+
+            int code = conn.getResponseCode();
+
+            if (code == 200) {
+                String redirectUrl = conn.getURL().toString();
+
+                checkDownloadFolder();
+
+                FileOutputStream outFile = new FileOutputStream(downloadFolder + "/" + fileName(redirectUrl));
+
+                outFile.getChannel().transferFrom(Channels.newChannel(conn.getInputStream()), 0, Long.MAX_VALUE);
+
+                outFile.close();
+
+                return parseVersion(redirectUrl);
+            } else if (code == 304)
+                // This version is latest.
+                return currentVersion;
+            else
+                throw new RuntimeException("Got unexpected response code. Response code: " + code);
+        }
+        catch (IOException e) {
+            throw new RuntimeException("Failed update ignite.", e);
+        }
+    }
+
+    /**
+     * @param version The current latest version.
+     * @return Ignite archive.
+     */
+    public String downloadIgnite(String version) {
+        try {
+            URL url = new URL(DIRECT_DOWNLOAD_LINK + version + ".zip");
+
+            HttpURLConnection conn = (HttpURLConnection)url.openConnection();
+
+            int code = conn.getResponseCode();
+
+            if (code == 200) {
+                checkDownloadFolder();
+
+                String fileName = fileName(url.toString());
+
+                FileOutputStream outFile = new FileOutputStream(downloadFolder + fileName);
+
+                outFile.getChannel().transferFrom(Channels.newChannel(conn.getInputStream()), 0, Long.MAX_VALUE);
+
+                outFile.close();
+
+                return fileName;
+            }
+            else
+                throw new RuntimeException("Got unexpected response code. Response code: " + code);
+        }
+        catch (IOException e) {
+            throw new RuntimeException("Failed update ignite.", e);
+        }
+    }
+
+
+    /**
+     * @return Download folder.
+     */
+    private File checkDownloadFolder() {
+        File file = new File(downloadFolder);
+
+        if (!file.exists())
+            file.mkdirs();
+
+        return file;
+    }
+
+    /**
+     * @param url URL.
+     * @return Ignite version.
+     */
+    public static String parseVersion(String url) {
+        String[] split = url.split("-");
+
+        return split[split.length - 1].replaceAll(".zip", "");
+    }
+
+    /**
+     * @param url URL.
+     * @return File name.
+     */
+    private static String fileName(String url) {
+        String[] split = url.split("/");
+
+        return split[split.length - 1];
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4b482187/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceController.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceController.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceController.java
new file mode 100644
index 0000000..5c9e693
--- /dev/null
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceController.java
@@ -0,0 +1,130 @@
+/*
+ * 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.mesos.resource;
+
+import javax.ws.rs.*;
+import javax.ws.rs.core.*;
+import java.io.*;
+
+/**
+ *
+ */
+@Path("/")
+public class ResourceController {
+    /** */
+    public static final String IGNITE_PREFIX = "/ignite/";
+
+    /** */
+    public static final String LIBS_PREFIX = "/libs/";
+
+    /** */
+    public static final String CONFIG_PREFIX = "/config/";
+
+    /** */
+    public static final String DEFAULT_CONFIG = CONFIG_PREFIX + "default/";
+
+    /** */
+    private String libsDir;
+
+    /** */
+    private String cfgPath;
+
+    /** */
+    private String igniteDir;
+
+    /**
+     * @param libsDir Path to directory with user libs.
+     * @param cfgPath Path to config file.
+     */
+    public ResourceController(String libsDir, String cfgPath, String igniteDir) {
+        this.libsDir = libsDir;
+        this.cfgPath = cfgPath;
+        this.igniteDir = igniteDir;
+    }
+
+    /**
+     *
+     * @param ignite
+     * @return
+     */
+    @GET
+    @Path(IGNITE_PREFIX + "{ignite-dist}")
+    public Response ignite(@PathParam("ignite-dist") String ignite) {
+        return handleRequest(new File(igniteDir + "/" + ignite), "application/zip-archive", ignite);
+    }
+
+    /**
+     *
+     * @param lib
+     * @return
+     */
+    @GET
+    @Path(LIBS_PREFIX + "{lib}")
+    public Response lib(@PathParam("lib") String lib) {
+        return handleRequest(new File(libsDir + "/" + lib), "application/java-archive", lib);
+    }
+
+    /**
+     *
+     * @param cfg
+     * @return
+     */
+    @GET
+    @Path(CONFIG_PREFIX + "{cfg}")
+    public Response config(@PathParam("cfg") String cfg) {
+        return handleRequest(new File(cfgPath), "application/xml", cfg);
+    }
+
+    /**
+     *
+     * @param cfg
+     * @return
+     */
+    @GET
+    @Path(DEFAULT_CONFIG + "{cfg}")
+    public Response defaultConfig(@PathParam("cfg") String cfg) {
+        return handleRequest(Thread.currentThread().getContextClassLoader().getResourceAsStream(cfg),
+            "application/xml", cfg);
+    }
+
+    /**
+     *
+     * @param resource File resource.
+     * @param type Type.
+     * @param attachmentName Attachment name.
+     * @return Http response.
+     */
+    private static Response handleRequest(File resource, String type, String attachmentName) {
+        final Response.ResponseBuilder builder = Response.ok(resource, type);
+        builder.header("Content-Disposition", String.format("attachment; filename=\"%s\"", attachmentName));
+        return builder.build();
+    }
+
+    /**
+     *
+     * @param resource File resource.
+     * @param type Type.
+     * @param attachmentName Attachment name.
+     * @return Http response.
+     */
+    private static Response handleRequest(InputStream resource, String type, String attachmentName) {
+        final Response.ResponseBuilder builder = Response.ok(resource, type);
+        builder.header("Content-Disposition", String.format("attachment; filename=\"%s\"", attachmentName));
+        return builder.build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4b482187/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceProvider.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceProvider.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceProvider.java
new file mode 100644
index 0000000..544b15c
--- /dev/null
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/resource/ResourceProvider.java
@@ -0,0 +1,120 @@
+/*
+ * 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.mesos.resource;
+
+import org.apache.ignite.mesos.*;
+
+import java.io.*;
+import java.util.*;
+
+import static org.apache.ignite.mesos.resource.ResourceController.*;
+
+/**
+ *
+ */
+public class ResourceProvider {
+    /** Ignite url. */
+    private String igniteUrl;
+
+    /** Resources. */
+    private Collection<String> libsUris;
+
+    /** Url config. */
+    private String configUrl;
+
+    /** Config name. */
+    private String configName;
+
+    /**
+     * @param properties Cluster properties.
+     * @param provider Ignite provider.
+     * @param baseUrl Base url.
+     */
+    public void init(ClusterProperties properties, IgniteProvider provider, String baseUrl) {
+        // Downloading ignite.
+        if (properties.igniteVer().equals(ClusterProperties.DEFAULT_IGNITE_VERSION))
+            igniteUrl = baseUrl + IGNITE_PREFIX + provider.getIgnite();
+        else
+            igniteUrl = baseUrl + IGNITE_PREFIX + provider.getIgnite(properties.igniteVer());
+
+        // Find all jar files into user folder.
+        if (properties.userLibs() != null && !properties.userLibs().isEmpty()) {
+            File libsDir = new File(properties.userLibs());
+
+            List<String> libs = new ArrayList<>();
+
+            if (libsDir.isDirectory()) {
+                File[] files = libsDir.listFiles();
+
+                if (files != null) {
+                    for (File lib : files) {
+                        if (lib.isFile() && lib.canRead() &&
+                            (lib.getName().endsWith(".jar") || lib.getName().endsWith(".JAR")))
+                            libs.add(baseUrl + LIBS_PREFIX + lib.getName());
+                    }
+                }
+            }
+
+            libsUris = libs.isEmpty() ? null : libs;
+        }
+
+        // Set configuration url.
+        if (properties.igniteCfg() != null) {
+            File cfg = new File(properties.igniteCfg());
+
+            if (cfg.isFile() && cfg.canRead()) {
+                configUrl = baseUrl + CONFIG_PREFIX + cfg.getName();
+
+                configName = cfg.getName();
+            }
+        }
+        else {
+            configName = "ignite-default-config.xml";
+
+            configUrl = baseUrl + DEFAULT_CONFIG + configName;
+        }
+    }
+
+    /**
+     * @return Config name.
+     */
+    public String configName() {
+        return configName;
+    }
+
+    /**
+     * @return Ignite url.
+     */
+    public String igniteUrl() {
+        return igniteUrl;
+    }
+
+    /**
+     * @return Urls to user's libs.
+     */
+    public Collection<String> resourceUrl() {
+        return libsUris;
+    }
+
+    /**
+     * @return Url to config file.
+     */
+    public String igniteConfigUrl() {
+        return configUrl;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4b482187/modules/mesos/src/main/resources/ignite-default-config.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/resources/ignite-default-config.xml b/modules/mesos/src/main/resources/ignite-default-config.xml
new file mode 100644
index 0000000..9fcce97
--- /dev/null
+++ b/modules/mesos/src/main/resources/ignite-default-config.xml
@@ -0,0 +1,35 @@
+<?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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+                            http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="shared" value="true"/>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4b482187/modules/mesos/src/main/resources/log4j2.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/resources/log4j2.xml b/modules/mesos/src/main/resources/log4j2.xml
new file mode 100644
index 0000000..d66a83f
--- /dev/null
+++ b/modules/mesos/src/main/resources/log4j2.xml
@@ -0,0 +1,35 @@
+<?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.
+-->
+
+<configuration strict="true" status="OFF" monitorInterval="30"  shutdownHook="disable">
+
+    <appenders>
+        <appender type="Console" name="STDOUT">
+            <layout type="PatternLayout" pattern="%d %p %C{1.} [%t] %m%n %ex"/>
+        </appender>
+    </appenders>
+
+    <loggers>
+        <root level="info">
+            <appender-ref ref="STDOUT"/>
+        </root>
+
+    </loggers>
+
+</configuration>
\ No newline at end of file



[12/37] incubator-ignite git commit: #IGNITE-857 WIP.

Posted by sb...@apache.org.
#IGNITE-857 WIP.


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

Branch: refs/heads/ignite-gg-10369
Commit: daac03eedbd47f615af06285e8ad6934a5586729
Parents: c462a0f
Author: nikolay tikhonov <nt...@gridgain.com>
Authored: Tue May 26 09:01:57 2015 +0300
Committer: nikolay tikhonov <nt...@gridgain.com>
Committed: Tue May 26 09:01:57 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/mesos/IgniteFramework.java | 19 +++++++++++++------
 1 file changed, 13 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/daac03ee/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
index 6cc43b1..b385bc9 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
@@ -23,6 +23,7 @@ import org.apache.mesos.*;
 import org.glassfish.grizzly.http.server.*;
 import org.glassfish.jersey.grizzly2.httpserver.*;
 import org.glassfish.jersey.server.*;
+import org.slf4j.*;
 
 import java.net.*;
 
@@ -30,7 +31,10 @@ import java.net.*;
  * Ignite mesos framework.
  */
 public class IgniteFramework {
+    /** */
+    public static final Logger log = LoggerFactory.getLogger(IgniteFramework.class);
 
+    /** Framework name. */
     public static final String IGNITE_FRAMEWORK_NAME = "Ignite";
 
     /**
@@ -48,11 +52,12 @@ public class IgniteFramework {
             .setFailoverTimeout(frameworkFailoverTimeout);
 
         if (System.getenv("MESOS_CHECKPOINT") != null) {
-            System.out.println("Enabling checkpoint for the framework");
+            log.info("Enabling checkpoint for the framework");
+
             frameworkBuilder.setCheckpoint(true);
         }
 
-        ClusterProperties clusterProps = ClusterProperties.from(args.length == 1 ? args[0] : null);
+        ClusterProperties clusterProps = ClusterProperties.from(args.length >= 1 ? args[0] : null);
 
         String baseUrl = String.format("http://%s:%d", clusterProps.httpServerHost(), clusterProps.httpServerPort());
 
@@ -60,7 +65,7 @@ public class IgniteFramework {
 
         ResourceConfig rc = new ResourceConfig()
             .registerInstances(new ResourceController(clusterProps.userLibs(), clusterProps.igniteCfg(),
-                    clusterProps.igniteWorkDir()));
+                clusterProps.igniteWorkDir()));
 
         HttpServer httpServer = GrizzlyHttpServerFactory.createHttpServer(httpServerBaseUri, rc);
 
@@ -76,15 +81,17 @@ public class IgniteFramework {
         // create the driver
         MesosSchedulerDriver driver;
         if (System.getenv("MESOS_AUTHENTICATE") != null) {
-            System.out.println("Enabling authentication for the framework");
+            log.info("Enabling authentication for the framework");
 
             if (System.getenv("DEFAULT_PRINCIPAL") == null) {
-                System.err.println("Expecting authentication principal in the environment");
+                log.error("Expecting authentication principal in the environment");
+
                 System.exit(1);
             }
 
             if (System.getenv("DEFAULT_SECRET") == null) {
-                System.err.println("Expecting authentication secret in the environment");
+                log.error("Expecting authentication secret in the environment");
+
                 System.exit(1);
             }
 


[07/37] incubator-ignite git commit: #IGNITE-857 Added resource limit.

Posted by sb...@apache.org.
#IGNITE-857 Added resource limit.


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

Branch: refs/heads/ignite-gg-10369
Commit: e320873828284fb86fb4d6e52cee98a6bb87b4af
Parents: 55c166a
Author: nikolay tikhonov <nt...@gridgain.com>
Authored: Tue May 19 19:42:22 2015 +0300
Committer: nikolay tikhonov <nt...@gridgain.com>
Committed: Tue May 19 19:42:22 2015 +0300

----------------------------------------------------------------------
 modules/mesos/pom.xml                           |   6 +-
 .../apache/ignite/mesos/ClusterResources.java   | 130 +++++++++++++++++++
 .../apache/ignite/mesos/IgniteFramework.java    |   4 +-
 .../apache/ignite/mesos/IgniteScheduler.java    | 120 ++++++++---------
 .../org/apache/ignite/mesos/IgniteTask.java     |  78 +++++++++++
 .../ignite/mesos/IgniteSchedulerSelfTest.java   |   1 -
 6 files changed, 272 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e3208738/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index ef73c0b..5ce3e5c 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -68,6 +68,11 @@
                     <descriptorRefs>
                         <descriptorRef>jar-with-dependencies</descriptorRef>
                     </descriptorRefs>
+                    <archive>
+                        <manifest>
+                            <mainClass>org.apache.ignite.mesos.IgniteFramework</mainClass>
+                        </manifest>
+                    </archive>
                 </configuration>
                 <executions>
                     <execution>
@@ -79,7 +84,6 @@
                     </execution>
                 </executions>
             </plugin>
-
         </plugins>
     </build>
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e3208738/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterResources.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterResources.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterResources.java
new file mode 100644
index 0000000..0a2193f
--- /dev/null
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterResources.java
@@ -0,0 +1,130 @@
+/*
+ * 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.mesos;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Cluster settings.
+ */
+public class ClusterResources {
+    /** Unlimited. */
+    public static final int DEFAULT_VALUE = -1;
+
+    /** */
+    public static final String IGNITE_RESOURCE_CPU_CORES = "IGNITE_RESOURCE_CPU_CORES";
+
+    /** CPU limit. */
+    private double cpu = DEFAULT_VALUE;
+
+    /** */
+    public static final String IGNITE_RESOURCE_MEM_MB = "IGNITE_RESOURCE_MEM_MB";
+
+    /** Memory limit. */
+    private double mem = DEFAULT_VALUE;
+
+    /** */
+    public static final String IGNITE_RESOURCE_DISK_MB = "IGNITE_RESOURCE_DISK_MB";
+
+    /** Disk space limit. */
+    private double disk = DEFAULT_VALUE;
+
+    /** */
+    public static final String IGNITE_RESOURCE_NODE_CNT = "IGNITE_RESOURCE_NODE_CNT";
+
+    /** Node count limit. */
+    private double nodeCnt = DEFAULT_VALUE;
+
+    /** */
+    public ClusterResources() {
+        // No-op.
+    }
+
+    /**
+     * @return CPU count limit.
+     */
+    public double cpus(){
+        return cpu;
+    }
+
+    /**
+     * @return mem limit.
+     */
+    public double memory() {
+        return mem;
+    }
+
+    /**
+     * @return disk limit.
+     */
+    public double disk() {
+        return disk;
+    }
+
+    /**
+     * @return instance count limit.
+     */
+    public double instances() {
+        return nodeCnt;
+    }
+
+    /**
+     * @param config path to config file.
+     * @return Cluster configuration.
+     */
+    public static ClusterResources from(String config) {
+        try {
+            Properties props = new Properties();
+
+            props.load(new FileInputStream(config));
+
+            ClusterResources resources = new ClusterResources();
+
+            resources.cpu = getProperty(IGNITE_RESOURCE_CPU_CORES, props);
+            resources.mem = getProperty(IGNITE_RESOURCE_MEM_MB, props);
+            resources.disk = getProperty(IGNITE_RESOURCE_DISK_MB, props);
+            resources.nodeCnt = getProperty(IGNITE_RESOURCE_NODE_CNT, props);
+
+            return resources;
+        }
+        catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * @param name Property name.
+     * @param fileProps Property file.
+     * @return Property value.
+     */
+    private static double getProperty(String name, Properties fileProps) {
+        if (fileProps.containsKey(name))
+            return Double.valueOf(fileProps.getProperty(name));
+
+        String property = System.getProperty(name);
+
+        if (property == null)
+            System.getenv(name);
+
+        if (property == null)
+            return DEFAULT_VALUE;
+
+        return Double.valueOf(property);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e3208738/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
index 5c556a1..3d309f3 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
@@ -25,7 +25,7 @@ import org.apache.mesos.*;
  */
 public class IgniteFramework {
     /**
-     * @param args Args
+     * @param args Args [host:port] [resource limit]
      */
     public static void main(String[] args) {
         checkArgs(args);
@@ -43,7 +43,7 @@ public class IgniteFramework {
         }
 
         // create the scheduler
-        final Scheduler scheduler = new IgniteScheduler();
+        final Scheduler scheduler = new IgniteScheduler(ClusterResources.from(args[1]));
 
         // create the driver
         MesosSchedulerDriver driver;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e3208738/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index 7b5623b..fcbab87 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -40,26 +40,39 @@ public class IgniteScheduler implements Scheduler {
     /** Mem. */
     public static final String MEM = "mem";
 
+    /** Disk. */
+    public static final String DISK = "disk";
+
     /** Default port range. */
     public static final String DEFAULT_PORT = ":47500..47510";
 
+    /** Min of memory required. */
+    public static final int MIN_MEMORY = 256;
+
     /** Delimiter to use in IP names. */
     public static final String DELIM = ",";
 
-    /** ID generator. */
-    private AtomicInteger taskIdGenerator = new AtomicInteger();
-
     /** Logger. */
     private static final Logger log = LoggerFactory.getLogger(IgniteScheduler.class);
 
-    /** Min of memory required. */
-    public static final int MIN_MEMORY = 256;
-
     /** Mutex. */
     private static final Object mux = new Object();
 
+    /** ID generator. */
+    private AtomicInteger taskIdGenerator = new AtomicInteger();
+
     /** Task on host. */
-    private ConcurrentMap<String, String> tasks = new ConcurrentHashMap<>();
+    private ConcurrentMap<String, IgniteTask> tasks = new ConcurrentHashMap<>();
+
+    /** Cluster resources. */
+    private ClusterResources clusterLimit;
+
+    /**
+     * @param clusterLimit Resources limit.
+     */
+    public IgniteScheduler(ClusterResources clusterLimit) {
+        this.clusterLimit = clusterLimit;
+    }
 
     /** {@inheritDoc} */
     @Override public void registered(SchedulerDriver schedulerDriver, Protos.FrameworkID frameworkID,
@@ -78,10 +91,10 @@ public class IgniteScheduler implements Scheduler {
             log.info("resourceOffers() with {} offers", offers.size());
 
             for (Protos.Offer offer : offers) {
-                Tuple<Double, Double> cpuMem = checkOffer(offer);
+                IgniteTask igniteTask = checkOffer(offer);
 
                 // Decline offer which doesn't match by mem or cpu.
-                if (cpuMem == null) {
+                if (igniteTask == null) {
                     schedulerDriver.declineOffer(offer.getId());
 
                     continue;
@@ -94,13 +107,13 @@ public class IgniteScheduler implements Scheduler {
                 log.info("Launching task {}", taskId.getValue());
 
                 // Create task to run.
-                Protos.TaskInfo task = createTask(offer, cpuMem, taskId);
+                Protos.TaskInfo task = createTask(offer, igniteTask, taskId);
 
                 schedulerDriver.launchTasks(Collections.singletonList(offer.getId()),
                     Collections.singletonList(task),
                     Protos.Filters.newBuilder().setRefuseSeconds(1).build());
 
-                tasks.put(taskId.getValue(), offer.getHostname());
+                tasks.put(taskId.getValue(), igniteTask);
             }
         }
     }
@@ -109,11 +122,11 @@ public class IgniteScheduler implements Scheduler {
      * Create Task.
      *
      * @param offer Offer.
-     * @param cpuMem Cpu and mem on slave.
+     * @param igniteTask Task description.
      * @param taskId Task id.
      * @return Task.
      */
-    protected Protos.TaskInfo createTask(Protos.Offer offer, Tuple<Double, Double> cpuMem, Protos.TaskID taskId) {
+    protected Protos.TaskInfo createTask(Protos.Offer offer, IgniteTask igniteTask, Protos.TaskID taskId) {
         // Docker image info.
         Protos.ContainerInfo.DockerInfo.Builder docker = Protos.ContainerInfo.DockerInfo.newBuilder()
             .setImage(IMAGE)
@@ -131,16 +144,16 @@ public class IgniteScheduler implements Scheduler {
             .addResources(Protos.Resource.newBuilder()
                 .setName(CPUS)
                 .setType(Protos.Value.Type.SCALAR)
-                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem.get1())))
+                .setScalar(Protos.Value.Scalar.newBuilder().setValue(igniteTask.cpuCores())))
             .addResources(Protos.Resource.newBuilder()
                 .setName(MEM)
                 .setType(Protos.Value.Type.SCALAR)
-                .setScalar(Protos.Value.Scalar.newBuilder().setValue(cpuMem.get2())))
+                .setScalar(Protos.Value.Scalar.newBuilder().setValue(igniteTask.mem())))
             .setContainer(cont)
             .setCommand(Protos.CommandInfo.newBuilder()
                 .setShell(false)
                 .addArguments(STARTUP_SCRIPT)
-                .addArguments(String.valueOf(cpuMem.get2().intValue()))
+                .addArguments(String.valueOf(igniteTask.mem()))
                 .addArguments(getAddress()))
             .build();
     }
@@ -154,8 +167,8 @@ public class IgniteScheduler implements Scheduler {
 
         StringBuilder sb = new StringBuilder();
 
-        for (String host : tasks.values())
-            sb.append(host).append(DEFAULT_PORT).append(DELIM);
+        for (IgniteTask task : tasks.values())
+            sb.append(task.host()).append(DEFAULT_PORT).append(DELIM);
 
         return sb.substring(0, sb.length() - 1);
     }
@@ -164,11 +177,15 @@ public class IgniteScheduler implements Scheduler {
      * Check slave resources and return resources infos.
      *
      * @param offer Offer request.
-     * @return Pair where first is cpus, second is memory.
+     * @return Ignite task description.
      */
-    private Tuple<Double, Double> checkOffer(Protos.Offer offer) {
-        double cpus = -1;
-        double mem = -1;
+    private IgniteTask checkOffer(Protos.Offer offer) {
+        if (checkLimit(clusterLimit.instances(), tasks.size()))
+            return null;
+
+        double cpus = -2;
+        double mem = -2;
+        double disk = -2;
 
         for (Protos.Resource resource : offer.getResourcesList()) {
             if (resource.getName().equals(CPUS)) {
@@ -183,17 +200,18 @@ public class IgniteScheduler implements Scheduler {
                 else
                     log.debug("Mem resource was not a scalar: " + resource.getType().toString());
             }
-            else if (resource.getName().equals("disk"))
-                log.debug("Ignoring disk resources from offer");
+            else if (resource.getType().equals(Protos.Value.Type.SCALAR))
+                disk = resource.getScalar().getValue();
+            else
+                log.debug("Disk resource was not a scalar: " + resource.getType().toString());
         }
 
-        if (cpus < 0)
-            log.debug("No cpus resource present");
-        if (mem < 0)
-            log.debug("No mem resource present");
+        if (checkLimit(clusterLimit.memory(), mem) &&
+            checkLimit(clusterLimit.cpus(), cpus) &&
+            checkLimit(clusterLimit.disk(), disk) &&
+            MIN_MEMORY <= mem)
 
-        if (cpus >= 1 && MIN_MEMORY <= mem)
-            return new Tuple<>(cpus, mem);
+            return new IgniteTask(offer.getHostname(), cpus, mem, disk);
         else {
             log.info("Offer not sufficient for slave request:\n" + offer.getResourcesList().toString() +
                 "\n" + offer.getAttributesList().toString() +
@@ -205,6 +223,15 @@ public class IgniteScheduler implements Scheduler {
         }
     }
 
+    /**
+     * @param limit Limit.
+     * @param value Value.
+     * @return {@code True} if limit isn't violated else {@code false}.
+     */
+    private boolean checkLimit(double limit, double value) {
+        return limit == ClusterResources.DEFAULT_VALUE || limit <= value;
+    }
+
     /** {@inheritDoc} */
     @Override public void offerRescinded(SchedulerDriver schedulerDriver, Protos.OfferID offerID) {
         log.info("offerRescinded()");
@@ -250,37 +277,4 @@ public class IgniteScheduler implements Scheduler {
     @Override public void error(SchedulerDriver schedulerDriver, String s) {
         log.error("error() {}", s);
     }
-
-    /**
-     * Tuple.
-     */
-    public static class Tuple<A, B> {
-        /** */
-        private final A val1;
-
-        /** */
-        private final B val2;
-
-        /**
-         *
-         */
-        public Tuple(A val1, B val2) {
-            this.val1 = val1;
-            this.val2 = val2;
-        }
-
-        /**
-         * @return val1
-         */
-        public A get1() {
-            return val1;
-        }
-
-        /**
-         * @return val2
-         */
-        public B get2() {
-            return val2;
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e3208738/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteTask.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteTask.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteTask.java
new file mode 100644
index 0000000..bad9996
--- /dev/null
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteTask.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.mesos;
+
+/**
+ * TODO
+ */
+public class IgniteTask {
+    /** */
+    public final String host;
+
+    /** */
+    public final double cpuCores;
+
+    /** */
+    public final double mem;
+
+    /** */
+    public final double disk;
+
+    /**
+     * Ignite launched task.
+     *
+     * @param host Host.
+     * @param cpuCores Cpu cores count.
+     * @param mem Memory.
+     * @param disk Disk.
+     */
+    public IgniteTask(String host, double cpuCores, double mem, double disk) {
+        this.host = host;
+        this.cpuCores = cpuCores;
+        this.mem = mem;
+        this.disk = disk;
+    }
+
+    /**
+     * @return Host.
+     */
+    public String host() {
+        return host;
+    }
+
+    /**
+     * @return Cores count.
+     */
+    public double cpuCores() {
+        return cpuCores;
+    }
+
+    /**
+     * @return Memory.
+     */
+    public double mem() {
+        return mem;
+    }
+
+    /**
+     * @return Disk.
+     */
+    public double disk() {
+        return disk;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e3208738/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java b/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
index 5534b2c..2c4b6ee 100644
--- a/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
+++ b/modules/mesos/src/test/java/org/apache/ignite/mesos/IgniteSchedulerSelfTest.java
@@ -160,6 +160,5 @@ public class IgniteSchedulerSelfTest extends TestCase {
             return null;
         }
 
-
     }
 }
\ No newline at end of file


[24/37] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-857' into ignite-857

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-857' into ignite-857


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

Branch: refs/heads/ignite-gg-10369
Commit: 488499523e38de41bddba825db1165dfbd93db25
Parents: 07a1095 a346a3c
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu May 28 11:16:25 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu May 28 11:16:25 2015 +0300

----------------------------------------------------------------------
 .gitignore                                      |    3 +-
 LICENSE                                         |  238 +
 LICENSE.txt                                     |  238 -
 NOTICE                                          |   12 +
 NOTICE.txt                                      |   12 -
 assembly/release-base.xml                       |    8 +-
 bin/ignite-schema-import.bat                    |    2 +-
 bin/ignite-schema-import.sh                     |    2 +-
 bin/ignite.bat                                  |    2 +-
 bin/ignite.sh                                   |    2 +-
 bin/ignitevisorcmd.bat                          |    2 +-
 bin/ignitevisorcmd.sh                           |    2 +-
 bin/include/build-classpath.bat                 |   46 +
 bin/include/build-classpath.sh                  |   71 +
 bin/include/functions.sh                        |    2 +-
 bin/include/target-classpath.bat                |   46 -
 bin/include/target-classpath.sh                 |   71 -
 dev-tools/build.gradle                          |   35 +-
 dev-tools/gradle/wrapper/gradle-wrapper.jar     |  Bin 0 -> 51017 bytes
 .../gradle/wrapper/gradle-wrapper.properties    |    6 +
 dev-tools/gradlew                               |  164 +
 dev-tools/slurp.sh                              |   76 +
 dev-tools/src/main/groovy/jiraslurp.groovy      |  570 +-
 .../streaming/StreamTransformerExample.java     |    4 +-
 .../streaming/StreamVisitorExample.java         |    4 +-
 .../ignite/examples/streaming/package-info.java |    1 -
 .../streaming/wordcount/CacheConfig.java        |    2 +-
 .../streaming/wordcount/QueryWords.java         |   12 +-
 .../streaming/wordcount/StreamWords.java        |   12 +-
 .../streaming/wordcount/package-info.java       |    1 -
 .../socket/WordsSocketStreamerClient.java       |   82 +
 .../socket/WordsSocketStreamerServer.java       |  124 +
 .../wordcount/socket/package-info.java          |   21 +
 .../ClientAbstractConnectivitySelfTest.java     |   14 +
 modules/cloud/pom.xml                           |    2 +-
 .../TcpDiscoveryCloudIpFinderSelfTest.java      |    2 -
 .../ignite/codegen/MessageCodeGenerator.java    |    4 +-
 modules/core/src/main/java/META-INF/LICENSE     |  238 +
 modules/core/src/main/java/META-INF/NOTICE      |   12 +
 .../internal/GridEventConsumeHandler.java       |   26 +
 .../apache/ignite/internal/IgniteKernal.java    |   26 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |  136 +-
 .../internal/direct/DirectByteBufferStream.java |    4 +-
 .../interop/InteropAwareEventFilter.java        |   37 +
 .../internal/interop/InteropBootstrap.java      |   34 +
 .../interop/InteropBootstrapFactory.java        |   39 +
 .../internal/interop/InteropException.java      |   71 +
 .../internal/interop/InteropIgnition.java       |  166 +
 .../interop/InteropLocalEventListener.java      |   28 +
 .../interop/InteropNoCallbackException.java     |   50 +
 .../internal/interop/InteropProcessor.java      |   36 +
 .../managers/communication/GridIoManager.java   |    6 +-
 .../communication/GridIoMessageFactory.java     |    4 +-
 .../GridLifecycleAwareMessageFilter.java        |    5 +-
 .../eventstorage/GridEventStorageManager.java   |   29 +-
 .../cache/DynamicCacheDescriptor.java           |   16 +-
 .../processors/cache/GridCacheAdapter.java      |  589 +-
 .../processors/cache/GridCacheContext.java      |    7 +
 .../cache/GridCacheEvictionManager.java         |    2 +-
 .../processors/cache/GridCacheIoManager.java    |    6 +-
 .../processors/cache/GridCacheMapEntry.java     |  123 +-
 .../processors/cache/GridCacheMvccManager.java  |    4 +-
 .../GridCachePartitionExchangeManager.java      |    7 +-
 .../processors/cache/GridCacheProcessor.java    |  189 +-
 .../processors/cache/GridCacheProxyImpl.java    |   24 -
 .../processors/cache/GridCacheSwapManager.java  |  215 +-
 .../processors/cache/GridCacheTtlManager.java   |   42 +-
 .../processors/cache/GridCacheUtils.java        |    5 +-
 .../processors/cache/IgniteCacheProxy.java      |    3 +-
 .../processors/cache/IgniteInternalCache.java   |   27 -
 ...ridCacheOptimisticCheckPreparedTxFuture.java |  434 --
 ...idCacheOptimisticCheckPreparedTxRequest.java |  232 -
 ...dCacheOptimisticCheckPreparedTxResponse.java |  179 -
 .../distributed/GridCacheTxRecoveryFuture.java  |  506 ++
 .../distributed/GridCacheTxRecoveryRequest.java |  261 +
 .../GridCacheTxRecoveryResponse.java            |  182 +
 .../GridDistributedCacheAdapter.java            |  210 +-
 .../distributed/GridDistributedLockRequest.java |  111 +-
 .../GridDistributedTxFinishRequest.java         |   70 +-
 .../distributed/GridDistributedTxMapping.java   |    5 +-
 .../GridDistributedTxPrepareRequest.java        |  112 +-
 .../GridDistributedTxRemoteAdapter.java         |   32 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   22 +-
 .../distributed/dht/GridDhtLocalPartition.java  |    2 +-
 .../distributed/dht/GridDhtLockFuture.java      |    2 -
 .../distributed/dht/GridDhtLockRequest.java     |   45 +-
 .../dht/GridDhtOffHeapCacheEntry.java           |   63 +
 .../dht/GridDhtTransactionalCacheAdapter.java   |   15 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |    3 -
 .../distributed/dht/GridDhtTxFinishRequest.java |   43 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   38 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   95 +-
 .../cache/distributed/dht/GridDhtTxMapping.java |    2 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |  100 +-
 .../dht/GridDhtTxPrepareRequest.java            |   60 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |    8 +-
 .../distributed/dht/GridNoStorageCacheMap.java  |    4 +-
 .../dht/GridPartitionedGetFuture.java           |    2 +-
 .../dht/atomic/GridDhtAtomicCache.java          |    7 +-
 .../atomic/GridDhtAtomicOffHeapCacheEntry.java  |   63 +
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   42 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |   36 +-
 .../dht/colocated/GridDhtColocatedCache.java    |    5 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   37 +-
 .../GridDhtColocatedOffHeapCacheEntry.java      |   63 +
 .../colocated/GridDhtDetachedCacheEntry.java    |    4 +-
 .../GridDhtPartitionsExchangeFuture.java        |    2 +-
 .../distributed/near/GridNearCacheAdapter.java  |   13 +-
 .../distributed/near/GridNearCacheEntry.java    |    6 +-
 .../distributed/near/GridNearLockFuture.java    |   11 -
 .../distributed/near/GridNearLockRequest.java   |   61 +-
 .../near/GridNearOffHeapCacheEntry.java         |   60 +
 .../near/GridNearOptimisticTxPrepareFuture.java |  768 ++
 .../GridNearPessimisticTxPrepareFuture.java     |  347 +
 .../near/GridNearTransactionalCache.java        |    4 -
 .../near/GridNearTxFinishRequest.java           |   28 +-
 .../cache/distributed/near/GridNearTxLocal.java |  109 +-
 .../near/GridNearTxPrepareFuture.java           | 1050 ---
 .../near/GridNearTxPrepareFutureAdapter.java    |  231 +
 .../near/GridNearTxPrepareRequest.java          |   52 +-
 .../near/GridNearTxPrepareResponse.java         |   28 +-
 .../distributed/near/GridNearTxRemote.java      |   24 +-
 .../processors/cache/local/GridLocalCache.java  |    8 +-
 .../cache/local/GridLocalCacheEntry.java        |   18 +
 .../local/atomic/GridLocalAtomicCache.java      |   27 +-
 .../cache/query/GridCacheQueryManager.java      |   21 +-
 .../cache/query/GridCacheSqlQuery.java          |    2 +-
 .../cache/query/GridCacheTwoStepQuery.java      |   17 +
 .../cache/transactions/IgniteInternalTx.java    |   19 +-
 .../transactions/IgniteTransactionsImpl.java    |    4 +-
 .../cache/transactions/IgniteTxAdapter.java     |   76 +-
 .../cache/transactions/IgniteTxEntry.java       |   48 +-
 .../cache/transactions/IgniteTxHandler.java     |  106 +-
 .../transactions/IgniteTxLocalAdapter.java      |  185 +-
 .../cache/transactions/IgniteTxLocalEx.java     |   21 +-
 .../cache/transactions/IgniteTxManager.java     |  271 +-
 .../datastreamer/DataStreamerImpl.java          |    2 +
 .../processors/igfs/IgfsDataManager.java        |    3 +
 .../processors/igfs/IgfsDeleteWorker.java       |    4 +
 .../processors/igfs/IgfsMetaManager.java        |    2 +-
 .../internal/processors/igfs/IgfsUtils.java     |   11 +-
 .../offheap/GridOffHeapProcessor.java           |   17 +
 .../processors/query/GridQueryIndexing.java     |   16 +
 .../processors/query/GridQueryProcessor.java    |   52 +-
 .../processors/resource/GridResourceField.java  |   16 +-
 .../processors/resource/GridResourceIoc.java    |  389 +-
 .../processors/resource/GridResourceMethod.java |   13 +
 .../resource/GridResourceProcessor.java         |   20 +-
 .../ignite/internal/util/IgniteUtils.java       |   28 +-
 .../util/lang/GridComputeJobWrapper.java        |   96 -
 .../ignite/internal/util/lang/GridFunc.java     | 7218 +++++-------------
 .../internal/util/nio/GridBufferedParser.java   |    4 -
 .../internal/util/nio/GridDelimitedParser.java  |   91 +
 .../util/nio/GridNioDelimitedBuffer.java        |  106 +
 .../util/offheap/GridOffHeapPartitionedMap.java |    9 +
 .../unsafe/GridUnsafePartitionedMap.java        |  155 +-
 .../internal/visor/query/VisorQueryArg.java     |   14 +-
 .../internal/visor/query/VisorQueryJob.java     |    2 +
 .../communication/tcp/TcpCommunicationSpi.java  |    2 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   48 +-
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |   10 +-
 .../org/apache/ignite/stream/StreamAdapter.java |  111 +
 .../ignite/stream/StreamTupleExtractor.java     |   33 +
 .../stream/socket/SocketMessageConverter.java   |   31 +
 .../ignite/stream/socket/SocketStreamer.java    |  218 +
 .../ignite/stream/socket/package-info.java      |   21 +
 .../resources/META-INF/classnames.properties    |   13 +-
 .../core/src/main/resources/ignite.properties   |    2 +-
 .../internal/GridUpdateNotifierSelfTest.java    |   21 +-
 .../processors/cache/CacheGetFromJobTest.java   |  110 +
 .../cache/CacheOffheapMapEntrySelfTest.java     |  168 +
 .../cache/CacheRemoveAllSelfTest.java           |   81 +
 .../GridCacheAbstractFailoverSelfTest.java      |   12 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  227 +-
 .../cache/GridCacheAbstractSelfTest.java        |    4 +-
 .../cache/IgniteCacheNearLockValueSelfTest.java |  145 +
 .../cache/OffHeapTieredTransactionSelfTest.java |  127 +
 .../GridCacheAbstractNodeRestartSelfTest.java   |  101 +-
 .../distributed/GridCacheLockAbstractTest.java  |    2 -
 .../distributed/IgniteTxGetAfterStopTest.java   |  131 +
 ...xOriginatingNodeFailureAbstractSelfTest.java |    2 +-
 ...icOffHeapTieredMultiNodeFullApiSelfTest.java |   43 +
 ...achePartitionedNearDisabledLockSelfTest.java |   47 +
 ...ionedNearDisabledOffHeapFullApiSelfTest.java |    8 +-
 ...DisabledOffHeapMultiNodeFullApiSelfTest.java |    8 +-
 ...abledOffHeapTieredAtomicFullApiSelfTest.java |   56 +
 ...earDisabledOffHeapTieredFullApiSelfTest.java |   33 +
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |   33 +
 ...rDisabledPrimaryNodeFailureRecoveryTest.java |   31 +
 ...rtitionedPrimaryNodeFailureRecoveryTest.java |   31 +
 ...woBackupsPrimaryNodeFailureRecoveryTest.java |   37 +
 ...ePrimaryNodeFailureRecoveryAbstractTest.java |  533 ++
 ...idCacheAtomicReplicatedFailoverSelfTest.java |    6 +
 ...CacheAtomicOffHeapTieredFullApiSelfTest.java |   32 +
 ...icOffHeapTieredMultiNodeFullApiSelfTest.java |   33 +
 ...yWriteOrderOffHeapTieredFullApiSelfTest.java |   33 +
 ...erOffHeapTieredMultiNodeFullApiSelfTest.java |   33 +
 ...achePartitionedMultiNodeFullApiSelfTest.java |   15 +-
 .../GridCachePartitionedNodeRestartTest.java    |    4 +-
 ...dCachePartitionedOffHeapFullApiSelfTest.java |    8 +-
 ...titionedOffHeapMultiNodeFullApiSelfTest.java |    8 +-
 ...PartitionedOffHeapTieredFullApiSelfTest.java |   32 +
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |   72 +
 ...ePartitionedOptimisticTxNodeRestartTest.java |    4 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |   25 +-
 .../near/IgniteCacheNearOnlyTxTest.java         |  190 +
 .../near/NoneRebalanceModeSelfTest.java         |   67 +
 .../GridCacheReplicatedFailoverSelfTest.java    |    6 +
 .../GridCacheReplicatedLockSelfTest.java        |    5 +
 .../GridCacheReplicatedNodeRestartSelfTest.java |   82 +
 ...idCacheReplicatedOffHeapFullApiSelfTest.java |    8 +-
 ...plicatedOffHeapMultiNodeFullApiSelfTest.java |    8 +-
 ...eReplicatedOffHeapTieredFullApiSelfTest.java |   33 +
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |   33 +
 .../RandomEvictionPolicyCacheSizeSelfTest.java  |   72 +
 .../IgniteCacheExpiryPolicyAbstractTest.java    |    2 +-
 .../IgniteCacheExpiryPolicyTestSuite.java       |    2 +
 .../expiry/IgniteCacheTtlCleanupSelfTest.java   |   85 +
 ...LocalAtomicOffHeapTieredFullApiSelfTest.java |   32 +
 .../GridCacheLocalIsolatedNodesSelfTest.java    |   18 +-
 .../GridCacheLocalOffHeapFullApiSelfTest.java   |    6 +-
 ...dCacheLocalOffHeapTieredFullApiSelfTest.java |   32 +
 .../igfs/IgfsClientCacheSelfTest.java           |  132 +
 .../processors/igfs/IgfsOneClientNodeTest.java  |  133 +
 .../processors/igfs/IgfsStreamsSelfTest.java    |    2 +-
 .../util/nio/GridNioDelimitedBufferTest.java    |  112 +
 .../discovery/tcp/TcpDiscoveryRestartTest.java  |  199 +
 .../stream/socket/SocketStreamerSelfTest.java   |  315 +
 .../ignite/stream/socket/package-info.java      |   21 +
 .../ignite/testsuites/IgniteBasicTestSuite.java |    1 +
 .../IgniteCacheEvictionSelfTestSuite.java       |    1 +
 .../IgniteCacheFailoverTestSuite.java           |   10 +-
 .../IgniteCacheFullApiSelfTestSuite.java        |   18 +
 .../testsuites/IgniteCacheRestartTestSuite.java |   11 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |    2 +
 .../testsuites/IgniteCacheTestSuite2.java       |    3 +
 .../testsuites/IgniteCacheTestSuite3.java       |    5 +-
 .../testsuites/IgniteCacheTestSuite4.java       |    4 +
 .../IgniteCacheTxRecoverySelfTestSuite.java     |    4 +
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |    3 +
 .../testsuites/IgniteStreamTestSuite.java       |   39 +
 .../testsuites/IgniteUtilSelfTestSuite.java     |    1 +
 modules/gce/pom.xml                             |    2 +-
 .../query/h2/GridH2IndexingGeoSelfTest.java     |   20 +-
 .../processors/hadoop/v2/HadoopV2Context.java   |   10 +-
 .../testsuites/IgniteHadoopTestSuite.java       |    2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   21 +
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |   18 +-
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |    4 +-
 .../query/h2/sql/GridSqlOperationType.java      |    2 +-
 .../processors/query/h2/sql/GridSqlQuery.java   |   20 +
 .../query/h2/sql/GridSqlQueryParser.java        |   10 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |   11 +-
 .../processors/query/h2/sql/GridSqlSelect.java  |    2 +-
 .../processors/query/h2/sql/GridSqlUnion.java   |    2 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |    3 +
 .../h2/twostep/GridReduceQueryExecutor.java     |  121 +-
 .../cache/GridCacheOffheapIndexGetSelfTest.java |  111 +
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |   21 +
 ...eQueryMultiThreadedOffHeapTiredSelfTest.java |   37 +
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   31 +-
 .../local/IgniteCacheLocalQuerySelfTest.java    |    6 +
 .../IgniteCacheQuerySelfTestSuite.java          |    1 +
 .../IgniteCacheWithIndexingTestSuite.java       |    2 +
 .../processors/cache/jta/CacheJtaManager.java   |    4 +-
 .../ignite/mesos/resource/JettyServer.java      |    4 +-
 .../ignite/scalar/ScalarConversions.scala       |    8 -
 .../scalar/tests/ScalarCacheQueriesSpec.scala   |  154 +-
 .../ignite/scalar/tests/ScalarCacheSpec.scala   |   23 +-
 .../scalar/tests/ScalarConversionsSpec.scala    |   43 +-
 .../scalar/tests/ScalarProjectionSpec.scala     |  128 +-
 .../scalar/tests/ScalarReturnableSpec.scala     |   41 +-
 .../ignite/schema/generator/CodeGenerator.java  |   41 +-
 modules/visor-console/pom.xml                   |    2 +-
 .../commands/cache/VisorCacheScanCommand.scala  |    2 +-
 .../visor/commands/node/VisorNodeCommand.scala  |    2 +-
 .../commands/tasks/VisorTasksCommand.scala      |    2 +-
 .../scala/org/apache/ignite/visor/visor.scala   |   63 +-
 .../ignite/visor/VisorRuntimeBaseSpec.scala     |    2 +-
 .../visor/commands/VisorArgListSpec.scala       |   60 +-
 .../commands/VisorFileNameCompleterSpec.scala   |   34 +-
 .../commands/ack/VisorAckCommandSpec.scala      |   20 +-
 .../commands/alert/VisorAlertCommandSpec.scala  |   68 +-
 .../cache/VisorCacheClearCommandSpec.scala      |   48 +-
 .../commands/cache/VisorCacheCommandSpec.scala  |   66 +-
 .../config/VisorConfigurationCommandSpec.scala  |    8 +-
 .../cswap/VisorCacheSwapCommandSpec.scala       |   24 +-
 .../deploy/VisorDeployCommandSpec.scala         |   10 +-
 .../disco/VisorDiscoveryCommandSpec.scala       |   46 +-
 .../events/VisorEventsCommandSpec.scala         |   28 +-
 .../visor/commands/gc/VisorGcCommandSpec.scala  |   30 +-
 .../commands/help/VisorHelpCommandSpec.scala    |   57 +-
 .../commands/kill/VisorKillCommandSpec.scala    |   58 +-
 .../commands/log/VisorLogCommandSpec.scala      |   10 +-
 .../commands/mem/VisorMemoryCommandSpec.scala   |   77 +-
 .../commands/node/VisorNodeCommandSpec.scala    |   22 +-
 .../commands/open/VisorOpenCommandSpec.scala    |   16 +-
 .../commands/ping/VisorPingCommandSpec.scala    |   16 +-
 .../commands/start/VisorStartCommandSpec.scala  |  126 +-
 .../commands/tasks/VisorTasksCommandSpec.scala  |  112 +-
 .../commands/top/VisorTopologyCommandSpec.scala |   52 +-
 .../commands/vvm/VisorVvmCommandSpec.scala      |   30 +-
 .../config/benchmark-client-mode.properties     |   89 +
 parent/pom.xml                                  |    5 +
 pom.xml                                         |  147 +-
 305 files changed, 14284 insertions(+), 11514 deletions(-)
----------------------------------------------------------------------



[27/37] incubator-ignite git commit: #IGNITE-857 Fixed review notes.

Posted by sb...@apache.org.
#IGNITE-857 Fixed review notes.


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

Branch: refs/heads/ignite-gg-10369
Commit: 1c04c8b454c5906bec83c0096e5157dec5ab4f85
Parents: 4884995
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu May 28 17:46:27 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu May 28 17:46:27 2015 +0300

----------------------------------------------------------------------
 modules/mesos/pom.xml                           | 13 ----
 .../apache/ignite/mesos/ClusterProperties.java  |  6 +-
 .../apache/ignite/mesos/IgniteFramework.java    |  4 +-
 .../apache/ignite/mesos/IgniteScheduler.java    | 68 ++++++++++++--------
 4 files changed, 46 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c04c8b4/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index 9079c66..d3a2cef 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -30,7 +30,6 @@
     <properties>
         <jetty.version>9.2.10.v20150310</jetty.version>
         <mesos.version>0.22.0</mesos.version>
-        <slf4j.version>1.7.12</slf4j.version>
         <log4j.version>2.0.2</log4j.version>
     </properties>
 
@@ -42,24 +41,12 @@
         </dependency>
 
         <dependency>
-            <groupId>org.slf4j</groupId>
-            <artifactId>slf4j-api</artifactId>
-            <version>${slf4j.version}</version>
-        </dependency>
-
-        <dependency>
             <groupId>org.apache.logging.log4j</groupId>
             <artifactId>log4j-core</artifactId>
             <version>${log4j.version}</version>
         </dependency>
 
         <dependency>
-            <groupId>org.apache.logging.log4j</groupId>
-            <artifactId>log4j-slf4j-impl</artifactId>
-            <version>${log4j.version}</version>
-        </dependency>
-
-        <dependency>
             <groupId>org.eclipse.jetty</groupId>
             <artifactId>jetty-server</artifactId>
             <version>${jetty.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c04c8b4/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
index 944735e..785eddc 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.mesos;
 
-import org.slf4j.*;
+import org.apache.logging.log4j.*;
 
 import java.io.*;
 import java.net.*;
@@ -29,7 +29,7 @@ import java.util.regex.*;
  */
 public class ClusterProperties {
     /** */
-    private static final Logger log = LoggerFactory.getLogger(ClusterProperties.class);
+    private static final Logger log = LogManager.getLogger(ClusterProperties.class);
 
     /** Unlimited. */
     public static final double UNLIMITED = Double.MAX_VALUE;
@@ -62,7 +62,7 @@ public class ClusterProperties {
     public static final String IGNITE_HTTP_SERVER_PORT = "IGNITE_HTTP_SERVER_PORT";
 
     /** */
-    public static final String DEFAULT_HTTP_SERVER_PORT = "47511";
+    public static final String DEFAULT_HTTP_SERVER_PORT = "48610";
 
     /** Http server host. */
     private int httpServerPort = Integer.valueOf(DEFAULT_HTTP_SERVER_PORT);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c04c8b4/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
index 154385b..c30d206 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteFramework.java
@@ -19,8 +19,8 @@ package org.apache.ignite.mesos;
 
 import com.google.protobuf.*;
 import org.apache.ignite.mesos.resource.*;
+import org.apache.logging.log4j.*;
 import org.apache.mesos.*;
-import org.slf4j.*;
 
 import java.net.*;
 
@@ -29,7 +29,7 @@ import java.net.*;
  */
 public class IgniteFramework {
     /** */
-    public static final Logger log = LoggerFactory.getLogger(IgniteFramework.class);
+    public static final Logger log = LogManager.getLogger(IgniteFramework.class);
 
     /** Framework name. */
     public static final String IGNITE_FRAMEWORK_NAME = "Ignite";

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c04c8b4/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index e833025..bca064f 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -18,8 +18,8 @@
 package org.apache.ignite.mesos;
 
 import org.apache.ignite.mesos.resource.*;
+import org.apache.logging.log4j.*;
 import org.apache.mesos.*;
-import org.slf4j.*;
 
 import java.util.*;
 import java.util.concurrent.atomic.*;
@@ -44,7 +44,7 @@ public class IgniteScheduler implements Scheduler {
     public static final String DELIM = ",";
 
     /** Logger. */
-    private static final Logger log = LoggerFactory.getLogger(IgniteScheduler.class);
+    private static final Logger log = LogManager.getLogger(IgniteScheduler.class);
 
     /** Mutex. */
     private static final Object mux = new Object();
@@ -72,10 +72,13 @@ public class IgniteScheduler implements Scheduler {
 
     /** {@inheritDoc} */
     @Override public void resourceOffers(SchedulerDriver schedulerDriver, List<Protos.Offer> offers) {
-        synchronized (mux) {
-            log.debug("Offers resources: {} ", offers.size());
+        log.debug("Offers resources: {} ", offers.size());
 
-            for (Protos.Offer offer : offers) {
+        for (Protos.Offer offer : offers) {
+            Protos.TaskID taskId;
+            Protos.TaskInfo task;
+
+            synchronized (mux) {
                 IgniteTask igniteTask = checkOffer(offer);
 
                 // Decline offer which doesn't match by mem or cpu.
@@ -86,19 +89,28 @@ public class IgniteScheduler implements Scheduler {
                 }
 
                 // Generate a unique task ID.
-                Protos.TaskID taskId = Protos.TaskID.newBuilder()
+                taskId = Protos.TaskID.newBuilder()
                     .setValue(Integer.toString(taskIdGenerator.incrementAndGet())).build();
 
-                log.info("Launching task: [{}]", igniteTask);
+                log.info("Launching task: {}", igniteTask);
 
                 // Create task to run.
-                Protos.TaskInfo task = createTask(offer, igniteTask, taskId);
+                task = createTask(offer, igniteTask, taskId);
+
+                tasks.put(taskId.getValue(), igniteTask);
+            }
 
+            try {
                 schedulerDriver.launchTasks(Collections.singletonList(offer.getId()),
                     Collections.singletonList(task),
                     Protos.Filters.newBuilder().setRefuseSeconds(1).build());
+            }
+            catch (Exception e) {
+                synchronized (mux) {
+                    tasks.remove(taskId.getValue());
+                }
 
-                tasks.put(taskId.getValue(), igniteTask);
+                log.error("Failed launch task. Task id: {}. Task info: {}", taskId, task);
             }
         }
     }
@@ -274,34 +286,36 @@ public class IgniteScheduler implements Scheduler {
     @Override public void statusUpdate(SchedulerDriver schedulerDriver, Protos.TaskStatus taskStatus) {
         final String taskId = taskStatus.getTaskId().getValue();
 
-        log.info("Received update event task: [{}] is in state: [{}]", taskId, taskStatus.getState());
+        log.info("Received update event task: {} is in state: {}", taskId, taskStatus.getState());
 
         if (taskStatus.getState().equals(Protos.TaskState.TASK_FAILED)
             || taskStatus.getState().equals(Protos.TaskState.TASK_ERROR)
             || taskStatus.getState().equals(Protos.TaskState.TASK_FINISHED)
             || taskStatus.getState().equals(Protos.TaskState.TASK_KILLED)
             || taskStatus.getState().equals(Protos.TaskState.TASK_LOST)) {
+            IgniteTask failedTask;
+
             synchronized (mux) {
-                IgniteTask failedTask = tasks.remove(taskId);
+                failedTask = tasks.remove(taskId);
+            }
 
-                if (failedTask != null) {
-                    List<Protos.Request> requests = new ArrayList<>();
+            if (failedTask != null) {
+                List<Protos.Request> requests = new ArrayList<>();
 
-                    Protos.Request request = Protos.Request.newBuilder()
-                        .addResources(Protos.Resource.newBuilder()
-                            .setType(Protos.Value.Type.SCALAR)
-                            .setName(MEM)
-                            .setScalar(Protos.Value.Scalar.newBuilder().setValue(failedTask.mem())))
-                        .addResources(Protos.Resource.newBuilder()
-                            .setType(Protos.Value.Type.SCALAR)
-                            .setName(CPU)
-                            .setScalar(Protos.Value.Scalar.newBuilder().setValue(failedTask.cpuCores())))
-                        .build();
+                Protos.Request request = Protos.Request.newBuilder()
+                    .addResources(Protos.Resource.newBuilder()
+                        .setType(Protos.Value.Type.SCALAR)
+                        .setName(MEM)
+                        .setScalar(Protos.Value.Scalar.newBuilder().setValue(failedTask.mem())))
+                    .addResources(Protos.Resource.newBuilder()
+                        .setType(Protos.Value.Type.SCALAR)
+                        .setName(CPU)
+                        .setScalar(Protos.Value.Scalar.newBuilder().setValue(failedTask.cpuCores())))
+                    .build();
 
-                    requests.add(request);
+                requests.add(request);
 
-                    schedulerDriver.requestResources(requests);
-                }
+                schedulerDriver.requestResources(requests);
             }
         }
     }
@@ -316,7 +330,7 @@ public class IgniteScheduler implements Scheduler {
     /** {@inheritDoc} */
     @Override public void registered(SchedulerDriver schedulerDriver, Protos.FrameworkID frameworkID,
         Protos.MasterInfo masterInfo) {
-        log.info("Scheduler registered. Master: [{}:{}], framework=[{}]", masterInfo.getIp(), masterInfo.getPort(),
+        log.info("Scheduler registered. Master: {}:{}, framework={}", masterInfo.getIp(), masterInfo.getPort(),
             frameworkID);
     }
 


[33/37] incubator-ignite git commit: # ignite-456: add comment

Posted by sb...@apache.org.
# ignite-456: add comment


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

Branch: refs/heads/ignite-gg-10369
Commit: 3d77af8cb13552dfb8a183029959bb7b148a4018
Parents: e16cb03
Author: ashutak <as...@gridgain.com>
Authored: Thu May 28 19:18:10 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Thu May 28 19:18:10 2015 +0300

----------------------------------------------------------------------
 dev-tools/src/main/groovy/jiraslurp.groovy | 3 +++
 scripts/git-patch-prop.sh                  | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d77af8c/dev-tools/src/main/groovy/jiraslurp.groovy
----------------------------------------------------------------------
diff --git a/dev-tools/src/main/groovy/jiraslurp.groovy b/dev-tools/src/main/groovy/jiraslurp.groovy
index 3547337..8498cf0 100644
--- a/dev-tools/src/main/groovy/jiraslurp.groovy
+++ b/dev-tools/src/main/groovy/jiraslurp.groovy
@@ -420,6 +420,9 @@ def runAllTestBuilds = {builds, jiraNum ->
             else {
                 postData = "<build>" +
                         "  <buildType id='$it'/>" +
+                        "  <comment>" +
+                        "    <text>Auto triggered build to validate last attached patch file at $jiraNum.</text>" +
+                        "  </comment>" +
                         "  <properties>" +
                         "    <property name='env.JIRA_NUM' value='$jiraNum'/>" +
                         "  </properties>" +

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d77af8c/scripts/git-patch-prop.sh
----------------------------------------------------------------------
diff --git a/scripts/git-patch-prop.sh b/scripts/git-patch-prop.sh
index 9c52583..c856fb4 100644
--- a/scripts/git-patch-prop.sh
+++ b/scripts/git-patch-prop.sh
@@ -19,6 +19,6 @@
 #
 # Git patch-file maker/applier properties.
 #
-IGNITE_DEFAULT_BRANCH='ignite-sprint-3'
+IGNITE_DEFAULT_BRANCH='ignite-sprint-5'
 
 PATCHES_HOME=${IGNITE_HOME}


[28/37] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-857' into ignite-857

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-857' into ignite-857


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

Branch: refs/heads/ignite-gg-10369
Commit: 62d15409cbe2af2f91092c9bd248cb1109218812
Parents: 1c04c8b a36214c
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu May 28 17:48:06 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu May 28 17:48:06 2015 +0300

----------------------------------------------------------------------
 dev-tools/gradle/wrapper/gradle-wrapper.jar     | Bin 51017 -> 0 bytes
 .../gradle/wrapper/gradle-wrapper.properties    |  18 +-
 dev-tools/gradlew                               | 163 ++++++++----
 dev-tools/slurp.sh                              |   2 +-
 dev-tools/src/main/groovy/jiraslurp.groovy      |  73 ++++--
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../src/main/java/org/apache/ignite/Ignite.java |   8 +-
 .../java/org/apache/ignite/IgniteServices.java  |   5 +-
 .../apache/ignite/internal/IgniteKernal.java    |   4 +-
 .../org/apache/ignite/services/Service.java     |   5 +-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  20 +-
 .../service/ClosureServiceClientsNodesTest.java | 245 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 modules/extdata/p2p/pom.xml                     |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |   4 +
 .../query/h2/sql/BaseH2CompareQueryTest.java    |  16 ++
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 .../apache/ignite/mesos/IgniteScheduler.java    |  10 +-
 .../ignite/mesos/IgniteSchedulerSelfTest.java   |   8 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 .../licenses/jcraft-revised-bsd.txt             |  28 ---
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |   2 +-
 49 files changed, 508 insertions(+), 167 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/62d15409/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --cc modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index bca064f,9507642..64e2931
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@@ -44,10 -44,10 +44,10 @@@ public class IgniteScheduler implement
      public static final String DELIM = ",";
  
      /** Logger. */
 -    private static final Logger log = LoggerFactory.getLogger(IgniteScheduler.class);
 +    private static final Logger log = LogManager.getLogger(IgniteScheduler.class);
  
      /** Mutex. */
-     private static final Object mux = new Object();
+     private final Object mux = new Object();
  
      /** ID generator. */
      private AtomicInteger taskIdGenerator = new AtomicInteger();


[29/37] incubator-ignite git commit: #IGNITE-857 Fixed review notes.

Posted by sb...@apache.org.
#IGNITE-857 Fixed review notes.


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

Branch: refs/heads/ignite-gg-10369
Commit: 3a0b5daf0e23b456f68a15b2faa7080af671fca2
Parents: 62d1540
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu May 28 18:02:28 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu May 28 18:02:28 2015 +0300

----------------------------------------------------------------------
 modules/mesos/README.txt                        | 26 +++++++++++++++++++-
 modules/mesos/pom.xml                           |  2 +-
 .../apache/ignite/mesos/ClusterProperties.java  |  8 +++---
 .../apache/ignite/mesos/IgniteScheduler.java    | 13 +++++-----
 4 files changed, 36 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3a0b5daf/modules/mesos/README.txt
----------------------------------------------------------------------
diff --git a/modules/mesos/README.txt b/modules/mesos/README.txt
index 084ccce..75a62f8 100644
--- a/modules/mesos/README.txt
+++ b/modules/mesos/README.txt
@@ -1,4 +1,28 @@
 Apache Ignite Mesos Module
 ------------------------
 
-TODO
\ No newline at end of file
+Apache Ignite Mesos module provides integration Apache Ignite with Apache Mesos.
+
+Importing Apache Ignite Mesos Module In Maven Project
+-------------------------------------
+
+If you are using Maven to manage dependencies of your project, you can add Cloud module
+dependency like this (replace '${ignite.version}' with actual Ignite version you are
+interested in):
+
+<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">
+    ...
+    <dependencies>
+        ...
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-mesos</artifactId>
+            <version>${ignite.version}</version>
+        </dependency>
+        ...
+    </dependencies>
+    ...
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3a0b5daf/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index d3a2cef..3cf9028 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -25,7 +25,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>ignite-mesos</artifactId>
-    <version>1.1.0-SNAPSHOT</version>
+    <version>1.1.1-SNAPSHOT</version>
 
     <properties>
         <jetty.version>9.2.10.v20150310</jetty.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3a0b5daf/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
index 785eddc..956dd6e 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
@@ -201,7 +201,7 @@ public class ClusterProperties {
     }
 
     /**
-     * Set CPU count limit.
+     * Sets CPU count limit.
      */
     public void cpus(double cpu) {
         this.cpu = cpu;
@@ -215,7 +215,7 @@ public class ClusterProperties {
     }
 
     /**
-     * Set CPU count limit.
+     * Sets CPU count limit.
      */
     public void cpusPerNode(double cpu) {
         this.cpuPerNode = cpu;
@@ -229,7 +229,7 @@ public class ClusterProperties {
     }
 
     /**
-     * Set mem limit.
+     * Sets mem limit.
      *
      * @param mem Memory.
      */
@@ -245,7 +245,7 @@ public class ClusterProperties {
     }
 
     /**
-     * Set mem limit.
+     * Sets mem limit.
      *
      * @param mem Memory.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3a0b5daf/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index 64e2931..263123c 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -77,9 +77,10 @@ public class IgniteScheduler implements Scheduler {
         for (Protos.Offer offer : offers) {
             Protos.TaskID taskId;
             Protos.TaskInfo task;
+            IgniteTask igniteTask;
 
             synchronized (mux) {
-                IgniteTask igniteTask = checkOffer(offer);
+                igniteTask = checkOffer(offer);
 
                 // Decline offer which doesn't match by mem or cpu.
                 if (igniteTask == null) {
@@ -96,8 +97,6 @@ public class IgniteScheduler implements Scheduler {
 
                 // Create task to run.
                 task = createTask(offer, igniteTask, taskId);
-
-                tasks.put(taskId.getValue(), igniteTask);
             }
 
             try {
@@ -106,12 +105,12 @@ public class IgniteScheduler implements Scheduler {
                     Protos.Filters.newBuilder().setRefuseSeconds(1).build());
             }
             catch (Exception e) {
-                synchronized (mux) {
-                    tasks.remove(taskId.getValue());
-                }
-
                 log.error("Failed launch task. Task id: {}. Task info: {}", taskId, task);
             }
+
+            synchronized (mux) {
+                tasks.put(taskId.getValue(), igniteTask);
+            }
         }
     }
 


[17/37] incubator-ignite git commit: #IGNITE-857 Code format

Posted by sb...@apache.org.
#IGNITE-857 Code format


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

Branch: refs/heads/ignite-gg-10369
Commit: 9bed1ffef6cdb4b69550e443637c61523733b0f5
Parents: b7ef1a8
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed May 27 15:40:04 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed May 27 15:40:04 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/mesos/ClusterProperties.java  |  2 +-
 .../apache/ignite/mesos/IgniteScheduler.java    | 58 +++++++++-----------
 2 files changed, 27 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9bed1ffe/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
index c23fcea..25216ca 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/ClusterProperties.java
@@ -164,7 +164,7 @@ public class ClusterProperties {
     private String igniteCfg = null;
 
     /** */
-    public static final String IGNITE_CONFIG_XML_URL = "IGNITE_XML_CONFIG_URL";
+    public static final String IGNITE_CONFIG_XML_URL = "IGNITE_CONFIG_XML_URL";
 
     /** Url to ignite config. */
     private String igniteCfgUrl = null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9bed1ffe/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
index 05d8c4f..9041629 100644
--- a/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
+++ b/modules/mesos/src/main/java/org/apache/ignite/mesos/IgniteScheduler.java
@@ -112,6 +112,9 @@ public class IgniteScheduler implements Scheduler {
      * @return Task.
      */
     protected Protos.TaskInfo createTask(Protos.Offer offer, IgniteTask igniteTask, Protos.TaskID taskId) {
+        String cfgUrl = clusterProps.igniteConfigUrl() != null ?
+            clusterProps.igniteConfigUrl() : resourceProvider.igniteConfigUrl();
+
         Protos.CommandInfo.Builder builder = Protos.CommandInfo.newBuilder()
             .setEnvironment(Protos.Environment.newBuilder().addVariables(Protos.Environment.Variable.newBuilder()
                 .setName("IGNITE_TCP_DISCOVERY_ADDRESSES")
@@ -121,39 +124,30 @@ public class IgniteScheduler implements Scheduler {
                     clusterProps.ignitePackageUrl() : resourceProvider.igniteUrl())
                 .setExtract(true))
             .addUris(Protos.CommandInfo.URI.newBuilder()
-                .setValue(clusterProps.igniteConfigUrl() != null ?
-                    clusterProps.igniteConfigUrl() : resourceProvider.igniteConfigUrl()));
-
-        if (resourceProvider.resourceUrl() != null || clusterProps.usersLibsUrl() != null) {
-            if (clusterProps.igniteConfigUrl() != null) {
-                builder.addUris(Protos.CommandInfo.URI.newBuilder()
-                    .setValue(clusterProps.igniteConfigUrl())
-                    .setExtract(true));
-
-                String[] split = clusterProps.igniteConfigUrl().split("/");
-
-                builder.setValue("cp *.jar ./gridgain-community-*/libs/ "
-                    + "&& ./gridgain-community-*/bin/ignite.sh "
-                    + split[split.length - 1]
-                    + " -J-Xmx" + String.valueOf((int)igniteTask.mem() + "m")
-                    + " -J-Xms" + String.valueOf((int)igniteTask.mem()) + "m");
-            }
-            else {
-                for (String url : resourceProvider.resourceUrl())
-                    builder.addUris(Protos.CommandInfo.URI.newBuilder().setValue(url));
-
-                builder.setValue("cp *.jar ./gridgain-community-*/libs/ "
-                    + "&& ./gridgain-community-*/bin/ignite.sh "
-                    + resourceProvider.configName()
-                    + " -J-Xmx" + String.valueOf((int)igniteTask.mem() + "m")
-                    + " -J-Xms" + String.valueOf((int)igniteTask.mem()) + "m");
-            }
+                .setValue(cfgUrl));
+
+        if (clusterProps.usersLibsUrl() != null)
+            builder.addUris(Protos.CommandInfo.URI.newBuilder()
+                .setValue(clusterProps.usersLibsUrl())
+                .setExtract(true));
+        else if (resourceProvider.resourceUrl() != null) {
+            for (String url : resourceProvider.resourceUrl())
+                builder.addUris(Protos.CommandInfo.URI.newBuilder().setValue(url));
+        }
+
+        String cfgName = resourceProvider.configName();
+
+        if (clusterProps.igniteConfigUrl() != null) {
+            String[] split = clusterProps.igniteConfigUrl().split("/");
+
+            cfgName = split[split.length - 1];
         }
-        else
-            builder.setValue("./gridgain-community-*/bin/ignite.sh "
-                + resourceProvider.configName()
-                + " -J-Xmx" + String.valueOf((int)igniteTask.mem() + "m")
-                + " -J-Xms" + String.valueOf((int)igniteTask.mem()) + "m");
+
+        builder.setValue("find . -maxdepth 1 -name \"*.jar\" -exec cp {} ./gridgain-community-*/libs/ \\; && "
+            + "./gridgain-community-*/bin/ignite.sh "
+            + cfgName
+            + " -J-Xmx" + String.valueOf((int)igniteTask.mem() + "m")
+            + " -J-Xms" + String.valueOf((int)igniteTask.mem()) + "m");
 
         return Protos.TaskInfo.newBuilder()
             .setName("Ignite node " + taskId.getValue())