You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2018/11/28 11:52:32 UTC

[01/50] [abbrv] ignite git commit: IGNITE-10323 Don't run deactivation process on inactive cluster - Fixes #5476.

Repository: ignite
Updated Branches:
  refs/heads/ignite-9720 ceacab0dd -> 04fdbc1af


IGNITE-10323 Don't run deactivation process on inactive cluster - Fixes #5476.

Signed-off-by: Pavel Kovalenko <jo...@gmail.com>


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

Branch: refs/heads/ignite-9720
Commit: 39b59d21e96bac51dca9a12702f1993ad96fead9
Parents: d45ea4b
Author: vd-pyatkov <vp...@gridgain.com>
Authored: Fri Nov 23 14:05:46 2018 +0300
Committer: Pavel Kovalenko <jo...@gmail.com>
Committed: Fri Nov 23 14:05:46 2018 +0300

----------------------------------------------------------------------
 .../cluster/GridClusterStateProcessor.java      |  3 +-
 ...erActivateDeactivateTestWithPersistence.java | 37 ++++++++++++++++++++
 2 files changed, 39 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/39b59d21/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
index 6d5d474..95ff770 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
@@ -849,7 +849,8 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
 
         DiscoveryDataClusterState curState = globalState;
 
-        if (!curState.transition() && curState.active() == activate && BaselineTopology.equals(curState.baselineTopology(), blt))
+        if (!curState.transition() && curState.active() == activate
+            && (!activate || BaselineTopology.equals(curState.baselineTopology(), blt)))
             return new GridFinishedFuture<>();
 
         GridChangeGlobalStateFuture startedFut = null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/39b59d21/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java
index 0972ea2..4dc4f27 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java
@@ -98,6 +98,43 @@ public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteCl
         activateCachesRestore(5, true);
     }
 
+    /**
+     * Test deactivation on cluster that is not yet activated.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDeactivateInactiveCluster() throws Exception {
+        ccfgs = new CacheConfiguration[] {
+            new CacheConfiguration<>("test_cache_1")
+                .setGroupName("test_cache"),
+            new CacheConfiguration<>("test_cache_2")
+                .setGroupName("test_cache")
+        };
+
+        Ignite ignite = startGrids(3);
+
+        ignite.cluster().active(true);
+
+        ignite.cache("test_cache_1")
+            .put("key1", "val1");
+        ignite.cache("test_cache_2")
+            .put("key1", "val1");
+
+        ignite.cluster().active(false);
+
+        assertFalse(ignite.cluster().active());
+
+        stopAllGrids();
+
+        ignite = startGrids(2);
+
+        assertFalse(ignite.cluster().active());
+
+        ignite.cluster().active(false);
+
+        assertFalse(ignite.cluster().active());
+    }
+
     /** */
     private Map<Integer, Integer> startGridsAndLoadData(int srvs) throws Exception {
         Ignite srv = startGrids(srvs);


[32/50] [abbrv] ignite git commit: IGNITE-9284: [ML] Add a Standard Scaler

Posted by ag...@apache.org.
IGNITE-9284: [ML] Add a Standard Scaler

this closes #4964


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

Branch: refs/heads/ignite-9720
Commit: 41f4225c4b2f2735bce4ce861b9a51afc80d5815
Parents: 46a84fd
Author: Ravil Galeyev <de...@yandex.ru>
Authored: Tue Nov 27 14:05:17 2018 +0300
Committer: Yury Babak <yb...@gridgain.com>
Committed: Tue Nov 27 14:05:17 2018 +0300

----------------------------------------------------------------------
 .../ml/preprocessing/StandardScalerExample.java |  84 +++++++++++++++
 .../standardscaling/StandardScalerData.java     |  56 ++++++++++
 .../StandardScalerPreprocessor.java             |  91 +++++++++++++++++
 .../standardscaling/StandardScalerTrainer.java  | 101 +++++++++++++++++++
 .../standardscaling/package-info.java           |  22 ++++
 .../StandardScalerPreprocessorTest.java         |  59 +++++++++++
 .../StandardScalerTrainerTest.java              |  85 ++++++++++++++++
 7 files changed, 498 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/41f4225c/examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/StandardScalerExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/StandardScalerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/StandardScalerExample.java
new file mode 100644
index 0000000..13d8635
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/StandardScalerExample.java
@@ -0,0 +1,84 @@
+/*
+ * 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.examples.ml.preprocessing;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.examples.ml.dataset.model.Person;
+import org.apache.ignite.examples.ml.util.DatasetHelper;
+import org.apache.ignite.ml.dataset.DatasetFactory;
+import org.apache.ignite.ml.dataset.primitive.SimpleDataset;
+import org.apache.ignite.ml.math.functions.IgniteBiFunction;
+import org.apache.ignite.ml.math.primitives.vector.Vector;
+import org.apache.ignite.ml.math.primitives.vector.VectorUtils;
+import org.apache.ignite.ml.preprocessing.standardscaling.StandardScalerTrainer;
+
+/**
+ * Example that shows how to use StandardScaler preprocessor to scale the given data.
+ *
+ * Machine learning preprocessors are built as a chain. Most often the first preprocessor is a feature extractor as
+ * shown in this example. The second preprocessor here is a {@code StandardScaler} preprocessor which is built on top of
+ * the feature extractor and represents a chain of itself and the underlying feature extractor.
+ */
+public class StandardScalerExample {
+    /** Run example. */
+    public static void main(String[] args) throws Exception {
+        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
+            System.out.println(">>> Standard scaler example started.");
+
+            IgniteCache<Integer, Person> persons = createCache(ignite);
+
+            // Defines first preprocessor that extracts features from an upstream data.
+            IgniteBiFunction<Integer, Person, Vector> featureExtractor = (k, v) -> VectorUtils.of(
+                v.getAge(),
+                v.getSalary()
+            );
+
+            // Defines second preprocessor that processes features.
+            IgniteBiFunction<Integer, Person, Vector> preprocessor = new StandardScalerTrainer<Integer, Person>()
+                .fit(ignite, persons, featureExtractor);
+
+            // Creates a cache based simple dataset containing features and providing standard dataset API.
+            try (SimpleDataset<?> dataset = DatasetFactory.createSimpleDataset(ignite, persons, preprocessor)) {
+                new DatasetHelper(dataset).describe();
+            }
+
+            System.out.println(">>> Standard scaler example completed.");
+        }
+    }
+
+    /** */
+    private static IgniteCache<Integer, Person> createCache(Ignite ignite) {
+        CacheConfiguration<Integer, Person> cacheConfiguration = new CacheConfiguration<>();
+
+        cacheConfiguration.setName("PERSONS");
+        cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, 2));
+
+        IgniteCache<Integer, Person> persons = ignite.createCache(cacheConfiguration);
+
+        persons.put(1, new Person("Mike", 42, 10000));
+        persons.put(2, new Person("John", 32, 64000));
+        persons.put(3, new Person("George", 53, 120000));
+        persons.put(4, new Person("Karl", 24, 70000));
+
+        return persons;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/41f4225c/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/standardscaling/StandardScalerData.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/standardscaling/StandardScalerData.java b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/standardscaling/StandardScalerData.java
new file mode 100644
index 0000000..f96dcc5
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/standardscaling/StandardScalerData.java
@@ -0,0 +1,56 @@
+/*
+ * 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.ml.preprocessing.standardscaling;
+
+/** A Service class for {@link StandardScalerTrainer} which used for sums holing. */
+public class StandardScalerData implements AutoCloseable {
+    /** Sum values of every feature. */
+    double[] sum;
+    /** Sum of squared values of every feature. */
+    double[] squaredSum;
+    /** Rows count */
+    long cnt;
+
+    /**
+     * Creates {@code StandardScalerData}.
+     *
+     * @param sum Sum values of every feature.
+     * @param squaredSum Sum of squared values of every feature.
+     * @param cnt Rows count.
+     */
+    public StandardScalerData(double[] sum, double[] squaredSum, long cnt) {
+        this.sum = sum;
+        this.squaredSum = squaredSum;
+        this.cnt = cnt;
+    }
+
+    /** Merges to current. */
+    StandardScalerData merge(StandardScalerData that) {
+        for (int i = 0; i < sum.length; i++) {
+            sum[i] += that.sum[i];
+            squaredSum[i] += that.squaredSum[i];
+        }
+
+        cnt += that.cnt;
+        return this;
+    }
+
+    /** */
+    @Override public void close() {
+        // Do nothing, GC will clean up.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/41f4225c/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/standardscaling/StandardScalerPreprocessor.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/standardscaling/StandardScalerPreprocessor.java b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/standardscaling/StandardScalerPreprocessor.java
new file mode 100644
index 0000000..293e86a
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/standardscaling/StandardScalerPreprocessor.java
@@ -0,0 +1,91 @@
+/*
+ * 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.ml.preprocessing.standardscaling;
+
+import org.apache.ignite.ml.math.functions.IgniteBiFunction;
+import org.apache.ignite.ml.math.primitives.vector.Vector;
+
+/**
+ * The preprocessing function that makes standard scaling, transforms features to make {@code mean} equal to {@code 0}
+ * and {@code variance} equal to {@code 1}. From mathematical point of view it's the following function which is applied
+ * to every element in a dataset:
+ *
+ * {@code a_i = (a_i - mean_i) / sigma_i for all i},
+ *
+ * where {@code i} is a number of column, {@code mean_i} is the mean value this column and {@code sigma_i} is the
+ * standard deviation in this column.
+ *
+ * @param <K> Type of a key in {@code upstream} data.
+ * @param <V> Type of a value in {@code upstream} data.
+ */
+public class StandardScalerPreprocessor<K, V> implements IgniteBiFunction<K, V, Vector> {
+    /** */
+    private static final long serialVersionUID = -5977957318991608203L;
+
+    /** Means for each column. */
+    private final double[] means;
+    /** Standard deviation for each column. */
+    private final double[] sigmas;
+
+    /** Base preprocessor. */
+    private final IgniteBiFunction<K, V, Vector> basePreprocessor;
+
+    /**
+     * Constructs a new instance of standardscaling preprocessor.
+     *
+     * @param means Means of each column.
+     * @param sigmas Standard deviations in each column.
+     * @param basePreprocessor Base preprocessor.
+     */
+    public StandardScalerPreprocessor(double[] means, double[] sigmas,
+        IgniteBiFunction<K, V, Vector> basePreprocessor) {
+        assert means.length == sigmas.length;
+
+        this.means = means;
+        this.sigmas = sigmas;
+        this.basePreprocessor = basePreprocessor;
+    }
+
+    /**
+     * Applies this preprocessor.
+     *
+     * @param k Key.
+     * @param v Value.
+     * @return Preprocessed row.
+     */
+    @Override public Vector apply(K k, V v) {
+        Vector res = basePreprocessor.apply(k, v);
+
+        assert res.size() == means.length;
+
+        for (int i = 0; i < res.size(); i++)
+            res.set(i, (res.get(i) - means[i]) / sigmas[i]);
+
+        return res;
+    }
+
+    /** */
+    public double[] getMeans() {
+        return means;
+    }
+
+    /** */
+    public double[] getSigmas() {
+        return sigmas;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/41f4225c/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/standardscaling/StandardScalerTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/standardscaling/StandardScalerTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/standardscaling/StandardScalerTrainer.java
new file mode 100644
index 0000000..3661772
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/standardscaling/StandardScalerTrainer.java
@@ -0,0 +1,101 @@
+/*
+ * 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.ml.preprocessing.standardscaling;
+
+import org.apache.ignite.ml.dataset.Dataset;
+import org.apache.ignite.ml.dataset.DatasetBuilder;
+import org.apache.ignite.ml.dataset.UpstreamEntry;
+import org.apache.ignite.ml.dataset.primitive.context.EmptyContext;
+import org.apache.ignite.ml.math.functions.IgniteBiFunction;
+import org.apache.ignite.ml.math.primitives.vector.Vector;
+import org.apache.ignite.ml.preprocessing.PreprocessingTrainer;
+
+/**
+ * Trainer of the standard scaler preprocessor.
+ *
+ * @param <K> Type of a key in {@code upstream} data.
+ * @param <V> Type of a value in {@code upstream} data.
+ */
+public class StandardScalerTrainer<K, V> implements PreprocessingTrainer<K, V, Vector, Vector> {
+    /** {@inheritDoc} */
+    @Override public StandardScalerPreprocessor<K, V> fit(DatasetBuilder<K, V> datasetBuilder,
+        IgniteBiFunction<K, V, Vector> basePreprocessor) {
+        StandardScalerData standardScalerData = computeSum(datasetBuilder, basePreprocessor);
+
+        int n = standardScalerData.sum.length;
+        long cnt = standardScalerData.cnt;
+        double[] mean = new double[n];
+        double[] sigma = new double[n];
+
+        for (int i = 0; i < n; i++) {
+            mean[i] = standardScalerData.sum[i] / cnt;
+            double variace = (standardScalerData.squaredSum[i] - Math.pow(standardScalerData.sum[i], 2) / cnt) / cnt;
+            sigma[i] = Math.sqrt(variace);
+        }
+        return new StandardScalerPreprocessor<>(mean, sigma, basePreprocessor);
+    }
+
+    /** Computes sum, squared sum and row count. */
+    private StandardScalerData computeSum(DatasetBuilder<K, V> datasetBuilder,
+        IgniteBiFunction<K, V, Vector> basePreprocessor) {
+        try (Dataset<EmptyContext, StandardScalerData> dataset = datasetBuilder.build(
+            (upstream, upstreamSize) -> new EmptyContext(),
+            (upstream, upstreamSize, ctx) -> {
+                double[] sum = null;
+                double[] squaredSum = null;
+                long cnt = 0;
+
+                while (upstream.hasNext()) {
+                    UpstreamEntry<K, V> entity = upstream.next();
+                    Vector row = basePreprocessor.apply(entity.getKey(), entity.getValue());
+
+                    if (sum == null) {
+                        sum = new double[row.size()];
+                        squaredSum = new double[row.size()];
+                    }
+                    else {
+                        assert sum.length == row.size() : "Base preprocessor must return exactly " + sum.length
+                            + " features";
+                    }
+
+                    ++cnt;
+                    for (int i = 0; i < row.size(); i++) {
+                        double x = row.get(i);
+                        sum[i] += x;
+                        squaredSum[i] += x * x;
+                    }
+                }
+                return new StandardScalerData(sum, squaredSum, cnt);
+            }
+        )) {
+
+            return dataset.compute(data -> data,
+                (a, b) -> {
+                    if (a == null)
+                        return b;
+                    if (b == null)
+                        return a;
+
+                    return a.merge(b);
+                });
+        }
+        catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/41f4225c/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/standardscaling/package-info.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/standardscaling/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/standardscaling/package-info.java
new file mode 100644
index 0000000..5f5de3b
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/standardscaling/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 Standard scaler preprocessor.
+ */
+package org.apache.ignite.ml.preprocessing.standardscaling;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/41f4225c/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/standardscaling/StandardScalerPreprocessorTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/standardscaling/StandardScalerPreprocessorTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/standardscaling/StandardScalerPreprocessorTest.java
new file mode 100644
index 0000000..3c325b3
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/standardscaling/StandardScalerPreprocessorTest.java
@@ -0,0 +1,59 @@
+/*
+ * 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.ml.preprocessing.standardscaling;
+
+import org.apache.ignite.ml.math.primitives.vector.Vector;
+import org.apache.ignite.ml.math.primitives.vector.VectorUtils;
+import org.junit.Test;
+
+import static org.junit.Assert.assertArrayEquals;
+
+/**
+ * Tests for {@link StandardScalerPreprocessor}.
+ */
+public class StandardScalerPreprocessorTest {
+
+    /** Test {@code apply()} method. */
+    @Test
+    public void testApply() {
+        double[][] inputData = new double[][] {
+            {0, 2., 4., .1},
+            {0, 1., -18., 2.2},
+            {1, 4., 10., -.1},
+            {1, 0., 22., 1.3}
+        };
+        double[] means = new double[] {0.5, 1.75, 4.5, 0.875};
+        double[] sigmas = new double[] {0.5, 1.47901995, 14.51723114, 0.93374247};
+
+        StandardScalerPreprocessor<Integer, Vector> preprocessor = new StandardScalerPreprocessor<>(
+            means,
+            sigmas,
+            (k, v) -> v
+        );
+
+        double[][] expectedData = new double[][] {
+            {-1., 0.16903085, -0.03444183, -0.82999331},
+            {-1., -0.50709255, -1.54988233, 1.41902081},
+            {1., 1.52127766, 0.37886012, -1.04418513},
+            {1., -1.18321596, 1.20546403, 0.45515762}
+        };
+
+        for (int i = 0; i < inputData.length; i++)
+            assertArrayEquals(expectedData[i], preprocessor.apply(i, VectorUtils.of(inputData[i])).asArray(), 1e-8);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/41f4225c/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/standardscaling/StandardScalerTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/standardscaling/StandardScalerTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/standardscaling/StandardScalerTrainerTest.java
new file mode 100644
index 0000000..679cc48
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/standardscaling/StandardScalerTrainerTest.java
@@ -0,0 +1,85 @@
+/*
+ * 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.ml.preprocessing.standardscaling;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.ignite.ml.common.TrainerTest;
+import org.apache.ignite.ml.dataset.DatasetBuilder;
+import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder;
+import org.apache.ignite.ml.math.primitives.vector.Vector;
+import org.apache.ignite.ml.math.primitives.vector.VectorUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertArrayEquals;
+
+/**
+ * Tests for {@link StandardScalerTrainer}.
+ */
+public class StandardScalerTrainerTest extends TrainerTest {
+
+    /** Data. */
+    private DatasetBuilder<Integer, Vector> datasetBuilder;
+
+    /** Trainer to be tested. */
+    private StandardScalerTrainer<Integer, Vector> standardizationTrainer;
+
+    /** */
+    @Before
+    public void prepareDataset() {
+        Map<Integer, Vector> data = new HashMap<>();
+        data.put(1, VectorUtils.of(0, 2., 4., .1));
+        data.put(2, VectorUtils.of(0, 1., -18., 2.2));
+        data.put(3, VectorUtils.of(1, 4., 10., -.1));
+        data.put(4, VectorUtils.of(1, 0., 22., 1.3));
+        datasetBuilder = new LocalDatasetBuilder<>(data, parts);
+    }
+
+    /** */
+    @Before
+    public void createTrainer() {
+        standardizationTrainer = new StandardScalerTrainer<>();
+    }
+
+    /** Test {@code fit()} method. */
+    @Test
+    public void testCalculatesCorrectMeans() {
+        double[] expectedMeans = new double[] {0.5, 1.75, 4.5, 0.875};
+
+        StandardScalerPreprocessor<Integer, Vector> preprocessor = standardizationTrainer.fit(
+            datasetBuilder,
+            (k, v) -> v
+        );
+
+        assertArrayEquals(expectedMeans, preprocessor.getMeans(), 1e-8);
+    }
+
+    /** Test {@code fit()} method. */
+    @Test
+    public void testCalculatesCorrectStandardDeviations() {
+        double[] expectedSigmas = new double[] {0.5, 1.47901995, 14.51723114, 0.93374247};
+
+        StandardScalerPreprocessor<Integer, Vector> preprocessor = standardizationTrainer.fit(
+            datasetBuilder,
+            (k, v) -> v
+        );
+
+        assertArrayEquals(expectedSigmas, preprocessor.getSigmas(), 1e-8);
+    }
+}


[26/50] [abbrv] ignite git commit: IGNITE-10399 Remove unused import. - Fixes #5493.

Posted by ag...@apache.org.
IGNITE-10399 Remove unused import. - Fixes #5493.

Signed-off-by: Nikolay Izhikov <ni...@apache.org>


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

Branch: refs/heads/ignite-9720
Commit: 25c41fa1de910a0eab29b2e9b3c53ca3307d3a97
Parents: 507aeb2
Author: Sergey Antonov <an...@gmail.com>
Authored: Mon Nov 26 22:43:55 2018 +0300
Committer: Nikolay Izhikov <ni...@apache.org>
Committed: Mon Nov 26 22:43:55 2018 +0300

----------------------------------------------------------------------
 .../processors/cache/persistence/pagemem/PageMemoryImpl.java        | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/25c41fa1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
index 03f66c0..f6aa059 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
@@ -85,7 +85,6 @@ import org.apache.ignite.internal.util.offheap.GridOffHeapOutOfMemoryException;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.spi.encryption.EncryptionSpi;
 import org.apache.ignite.spi.encryption.noop.NoopEncryptionSpi;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;


[04/50] [abbrv] ignite git commit: IGNITE-10335: move ML examples datasets files to resources

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/titanic_10_rows.csv
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/titanic_10_rows.csv b/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/titanic_10_rows.csv
deleted file mode 100644
index 8146db3..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/titanic_10_rows.csv
+++ /dev/null
@@ -1,11 +0,0 @@
-pclass;survived;name;sex;age;sibsp;parch;ticket;fare;cabin;embarked;boat;body;homedest
-1;1;Allen, Miss. Elisabeth Walton;;29;;;24160;211,3375;B5;;2;;St Louis, MO
-1;1;Allison, Master. Hudson Trevor;male;0,9167;1;2;113781;151,55;C22 C26;S;11;;Montreal, PQ / Chesterville, ON
-1;0;Allison, Miss. Helen Loraine;female;2;1;2;113781;151,55;C22 C26;S;;;Montreal, PQ / Chesterville, ON
-1;0;Allison, Mr. Hudson Joshua Creighton;male;30;1;2;113781;151,55;C22 C26;S;;135;Montreal, PQ / Chesterville, ON
-1;0;Allison, Mrs. Hudson J C (Bessie Waldo Daniels);female;25;1;2;113781;151,55;C22 C26;S;;;Montreal, PQ / Chesterville, ON
-1;1;Anderson, Mr. Harry;male;48;0;0;19952;26,55;E12;S;3;;New York, NY
-1;1;Andrews, Miss. Kornelia Theodosia;female;63;1;0;13502;77,9583;D7;S;10;;Hudson, NY
-1;0;Andrews, Mr. Thomas Jr;male;39;0;0;112050;0;A36;S;;;Belfast, NI
-1;1;Appleton, Mrs. Edward Dale (Charlotte Lamson);female;53;2;0;11769;51,4792;C101;S;D;;Bayside, Queens, NY
-1;0;Artagaveytia, Mr. Ramon;male;71;0;0;PC 17609;49,5042;;C;;22;Montevideo, Uruguay

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/two_classed_iris.csv
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/two_classed_iris.csv b/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/two_classed_iris.csv
deleted file mode 100644
index 872c699..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/two_classed_iris.csv
+++ /dev/null
@@ -1,100 +0,0 @@
-0	5.1	3.5	1.4	0.2
-0	4.9	3	1.4	0.2
-0	4.7	3.2	1.3	0.2
-0	4.6	3.1	1.5	0.2
-0	5	3.6	1.4	0.2
-0	5.4	3.9	1.7	0.4
-0	4.6	3.4	1.4	0.3
-0	5	3.4	1.5	0.2
-0	4.4	2.9	1.4	0.2
-0	4.9	3.1	1.5	0.1
-0	5.4	3.7	1.5	0.2
-0	4.8	3.4	1.6	0.2
-0	4.8	3	1.4	0.1
-0	4.3	3	1.1	0.1
-0	5.8	4	1.2	0.2
-0	5.7	4.4	1.5	0.4
-0	5.4	3.9	1.3	0.4
-0	5.1	3.5	1.4	0.3
-0	5.7	3.8	1.7	0.3
-0	5.1	3.8	1.5	0.3
-0	5.4	3.4	1.7	0.2
-0	5.1	3.7	1.5	0.4
-0	4.6	3.6	1	0.2
-0	5.1	3.3	1.7	0.5
-0	4.8	3.4	1.9	0.2
-0	5	3	1.6	0.2
-0	5	3.4	1.6	0.4
-0	5.2	3.5	1.5	0.2
-0	5.2	3.4	1.4	0.2
-0	4.7	3.2	1.6	0.2
-0	4.8	3.1	1.6	0.2
-0	5.4	3.4	1.5	0.4
-0	5.2	4.1	1.5	0.1
-0	5.5	4.2	1.4	0.2
-0	4.9	3.1	1.5	0.1
-0	5	3.2	1.2	0.2
-0	5.5	3.5	1.3	0.2
-0	4.9	3.1	1.5	0.1
-0	4.4	3	1.3	0.2
-0	5.1	3.4	1.5	0.2
-0	5	3.5	1.3	0.3
-0	4.5	2.3	1.3	0.3
-0	4.4	3.2	1.3	0.2
-0	5	3.5	1.6	0.6
-0	5.1	3.8	1.9	0.4
-0	4.8	3	1.4	0.3
-0	5.1	3.8	1.6	0.2
-0	4.6	3.2	1.4	0.2
-0	5.3	3.7	1.5	0.2
-0	5	3.3	1.4	0.2
-1	7	3.2	4.7	1.4
-1	6.4	3.2	4.5	1.5
-1	6.9	3.1	4.9	1.5
-1	5.5	2.3	4	1.3
-1	6.5	2.8	4.6	1.5
-1	5.7	2.8	4.5	1.3
-1	6.3	3.3	4.7	1.6
-1	4.9	2.4	3.3	1
-1	6.6	2.9	4.6	1.3
-1	5.2	2.7	3.9	1.4
-1	5	2	3.5	1
-1	5.9	3	4.2	1.5
-1	6	2.2	4	1
-1	6.1	2.9	4.7	1.4
-1	5.6	2.9	3.6	1.3
-1	6.7	3.1	4.4	1.4
-1	5.6	3	4.5	1.5
-1	5.8	2.7	4.1	1
-1	6.2	2.2	4.5	1.5
-1	5.6	2.5	3.9	1.1
-1	5.9	3.2	4.8	1.8
-1	6.1	2.8	4	1.3
-1	6.3	2.5	4.9	1.5
-1	6.1	2.8	4.7	1.2
-1	6.4	2.9	4.3	1.3
-1	6.6	3	4.4	1.4
-1	6.8	2.8	4.8	1.4
-1	6.7	3	5	1.7
-1	6	2.9	4.5	1.5
-1	5.7	2.6	3.5	1
-1	5.5	2.4	3.8	1.1
-1	5.5	2.4	3.7	1
-1	5.8	2.7	3.9	1.2
-1	6	2.7	5.1	1.6
-1	5.4	3	4.5	1.5
-1	6	3.4	4.5	1.6
-1	6.7	3.1	4.7	1.5
-1	6.3	2.3	4.4	1.3
-1	5.6	3	4.1	1.3
-1	5.5	2.5	4	1.3
-1	5.5	2.6	4.4	1.2
-1	6.1	3	4.6	1.4
-1	5.8	2.6	4	1.2
-1	5	2.3	3.3	1
-1	5.6	2.7	4.2	1.3
-1	5.7	3	4.2	1.2
-1	5.7	2.9	4.2	1.3
-1	6.2	2.9	4.3	1.3
-1	5.1	2.5	3	1.1
-1	5.7	2.8	4.1	1.3

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/wine.txt
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/wine.txt b/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/wine.txt
deleted file mode 100644
index a0b3962..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/wine.txt
+++ /dev/null
@@ -1,178 +0,0 @@
-1,14.23,1.71,2.43,15.6,127,2.8,3.06,.28,2.29,5.64,1.04,3.92,1065
-1,13.2,1.78,2.14,11.2,100,2.65,2.76,.26,1.28,4.38,1.05,3.4,1050
-1,13.16,2.36,2.67,18.6,101,2.8,3.24,.3,2.81,5.68,1.03,3.17,1185
-1,14.37,1.95,2.5,16.8,113,3.85,3.49,.24,2.18,7.8,.86,3.45,1480
-1,13.24,2.59,2.87,21,118,2.8,2.69,.39,1.82,4.32,1.04,2.93,735
-1,14.2,1.76,2.45,15.2,112,3.27,3.39,.34,1.97,6.75,1.05,2.85,1450
-1,14.39,1.87,2.45,14.6,96,2.5,2.52,.3,1.98,5.25,1.02,3.58,1290
-1,14.06,2.15,2.61,17.6,121,2.6,2.51,.31,1.25,5.05,1.06,3.58,1295
-1,14.83,1.64,2.17,14,97,2.8,2.98,.29,1.98,5.2,1.08,2.85,1045
-1,13.86,1.35,2.27,16,98,2.98,3.15,.22,1.85,7.22,1.01,3.55,1045
-1,14.1,2.16,2.3,18,105,2.95,3.32,.22,2.38,5.75,1.25,3.17,1510
-1,14.12,1.48,2.32,16.8,95,2.2,2.43,.26,1.57,5,1.17,2.82,1280
-1,13.75,1.73,2.41,16,89,2.6,2.76,.29,1.81,5.6,1.15,2.9,1320
-1,14.75,1.73,2.39,11.4,91,3.1,3.69,.43,2.81,5.4,1.25,2.73,1150
-1,14.38,1.87,2.38,12,102,3.3,3.64,.29,2.96,7.5,1.2,3,1547
-1,13.63,1.81,2.7,17.2,112,2.85,2.91,.3,1.46,7.3,1.28,2.88,1310
-1,14.3,1.92,2.72,20,120,2.8,3.14,.33,1.97,6.2,1.07,2.65,1280
-1,13.83,1.57,2.62,20,115,2.95,3.4,.4,1.72,6.6,1.13,2.57,1130
-1,14.19,1.59,2.48,16.5,108,3.3,3.93,.32,1.86,8.7,1.23,2.82,1680
-1,13.64,3.1,2.56,15.2,116,2.7,3.03,.17,1.66,5.1,.96,3.36,845
-1,14.06,1.63,2.28,16,126,3,3.17,.24,2.1,5.65,1.09,3.71,780
-1,12.93,3.8,2.65,18.6,102,2.41,2.41,.25,1.98,4.5,1.03,3.52,770
-1,13.71,1.86,2.36,16.6,101,2.61,2.88,.27,1.69,3.8,1.11,4,1035
-1,12.85,1.6,2.52,17.8,95,2.48,2.37,.26,1.46,3.93,1.09,3.63,1015
-1,13.5,1.81,2.61,20,96,2.53,2.61,.28,1.66,3.52,1.12,3.82,845
-1,13.05,2.05,3.22,25,124,2.63,2.68,.47,1.92,3.58,1.13,3.2,830
-1,13.39,1.77,2.62,16.1,93,2.85,2.94,.34,1.45,4.8,.92,3.22,1195
-1,13.3,1.72,2.14,17,94,2.4,2.19,.27,1.35,3.95,1.02,2.77,1285
-1,13.87,1.9,2.8,19.4,107,2.95,2.97,.37,1.76,4.5,1.25,3.4,915
-1,14.02,1.68,2.21,16,96,2.65,2.33,.26,1.98,4.7,1.04,3.59,1035
-1,13.73,1.5,2.7,22.5,101,3,3.25,.29,2.38,5.7,1.19,2.71,1285
-1,13.58,1.66,2.36,19.1,106,2.86,3.19,.22,1.95,6.9,1.09,2.88,1515
-1,13.68,1.83,2.36,17.2,104,2.42,2.69,.42,1.97,3.84,1.23,2.87,990
-1,13.76,1.53,2.7,19.5,132,2.95,2.74,.5,1.35,5.4,1.25,3,1235
-1,13.51,1.8,2.65,19,110,2.35,2.53,.29,1.54,4.2,1.1,2.87,1095
-1,13.48,1.81,2.41,20.5,100,2.7,2.98,.26,1.86,5.1,1.04,3.47,920
-1,13.28,1.64,2.84,15.5,110,2.6,2.68,.34,1.36,4.6,1.09,2.78,880
-1,13.05,1.65,2.55,18,98,2.45,2.43,.29,1.44,4.25,1.12,2.51,1105
-1,13.07,1.5,2.1,15.5,98,2.4,2.64,.28,1.37,3.7,1.18,2.69,1020
-1,14.22,3.99,2.51,13.2,128,3,3.04,.2,2.08,5.1,.89,3.53,760
-1,13.56,1.71,2.31,16.2,117,3.15,3.29,.34,2.34,6.13,.95,3.38,795
-1,13.41,3.84,2.12,18.8,90,2.45,2.68,.27,1.48,4.28,.91,3,1035
-1,13.88,1.89,2.59,15,101,3.25,3.56,.17,1.7,5.43,.88,3.56,1095
-1,13.24,3.98,2.29,17.5,103,2.64,2.63,.32,1.66,4.36,.82,3,680
-1,13.05,1.77,2.1,17,107,3,3,.28,2.03,5.04,.88,3.35,885
-1,14.21,4.04,2.44,18.9,111,2.85,2.65,.3,1.25,5.24,.87,3.33,1080
-1,14.38,3.59,2.28,16,102,3.25,3.17,.27,2.19,4.9,1.04,3.44,1065
-1,13.9,1.68,2.12,16,101,3.1,3.39,.21,2.14,6.1,.91,3.33,985
-1,14.1,2.02,2.4,18.8,103,2.75,2.92,.32,2.38,6.2,1.07,2.75,1060
-1,13.94,1.73,2.27,17.4,108,2.88,3.54,.32,2.08,8.90,1.12,3.1,1260
-1,13.05,1.73,2.04,12.4,92,2.72,3.27,.17,2.91,7.2,1.12,2.91,1150
-1,13.83,1.65,2.6,17.2,94,2.45,2.99,.22,2.29,5.6,1.24,3.37,1265
-1,13.82,1.75,2.42,14,111,3.88,3.74,.32,1.87,7.05,1.01,3.26,1190
-1,13.77,1.9,2.68,17.1,115,3,2.79,.39,1.68,6.3,1.13,2.93,1375
-1,13.74,1.67,2.25,16.4,118,2.6,2.9,.21,1.62,5.85,.92,3.2,1060
-1,13.56,1.73,2.46,20.5,116,2.96,2.78,.2,2.45,6.25,.98,3.03,1120
-1,14.22,1.7,2.3,16.3,118,3.2,3,.26,2.03,6.38,.94,3.31,970
-1,13.29,1.97,2.68,16.8,102,3,3.23,.31,1.66,6,1.07,2.84,1270
-1,13.72,1.43,2.5,16.7,108,3.4,3.67,.19,2.04,6.8,.89,2.87,1285
-2,12.37,.94,1.36,10.6,88,1.98,.57,.28,.42,1.95,1.05,1.82,520
-2,12.33,1.1,2.28,16,101,2.05,1.09,.63,.41,3.27,1.25,1.67,680
-2,12.64,1.36,2.02,16.8,100,2.02,1.41,.53,.62,5.75,.98,1.59,450
-2,13.67,1.25,1.92,18,94,2.1,1.79,.32,.73,3.8,1.23,2.46,630
-2,12.37,1.13,2.16,19,87,3.5,3.1,.19,1.87,4.45,1.22,2.87,420
-2,12.17,1.45,2.53,19,104,1.89,1.75,.45,1.03,2.95,1.45,2.23,355
-2,12.37,1.21,2.56,18.1,98,2.42,2.65,.37,2.08,4.6,1.19,2.3,678
-2,13.11,1.01,1.7,15,78,2.98,3.18,.26,2.28,5.3,1.12,3.18,502
-2,12.37,1.17,1.92,19.6,78,2.11,2,.27,1.04,4.68,1.12,3.48,510
-2,13.34,.94,2.36,17,110,2.53,1.3,.55,.42,3.17,1.02,1.93,750
-2,12.21,1.19,1.75,16.8,151,1.85,1.28,.14,2.5,2.85,1.28,3.07,718
-2,12.29,1.61,2.21,20.4,103,1.1,1.02,.37,1.46,3.05,.906,1.82,870
-2,13.86,1.51,2.67,25,86,2.95,2.86,.21,1.87,3.38,1.36,3.16,410
-2,13.49,1.66,2.24,24,87,1.88,1.84,.27,1.03,3.74,.98,2.78,472
-2,12.99,1.67,2.6,30,139,3.3,2.89,.21,1.96,3.35,1.31,3.5,985
-2,11.96,1.09,2.3,21,101,3.38,2.14,.13,1.65,3.21,.99,3.13,886
-2,11.66,1.88,1.92,16,97,1.61,1.57,.34,1.15,3.8,1.23,2.14,428
-2,13.03,.9,1.71,16,86,1.95,2.03,.24,1.46,4.6,1.19,2.48,392
-2,11.84,2.89,2.23,18,112,1.72,1.32,.43,.95,2.65,.96,2.52,500
-2,12.33,.99,1.95,14.8,136,1.9,1.85,.35,2.76,3.4,1.06,2.31,750
-2,12.7,3.87,2.4,23,101,2.83,2.55,.43,1.95,2.57,1.19,3.13,463
-2,12,.92,2,19,86,2.42,2.26,.3,1.43,2.5,1.38,3.12,278
-2,12.72,1.81,2.2,18.8,86,2.2,2.53,.26,1.77,3.9,1.16,3.14,714
-2,12.08,1.13,2.51,24,78,2,1.58,.4,1.4,2.2,1.31,2.72,630
-2,13.05,3.86,2.32,22.5,85,1.65,1.59,.61,1.62,4.8,.84,2.01,515
-2,11.84,.89,2.58,18,94,2.2,2.21,.22,2.35,3.05,.79,3.08,520
-2,12.67,.98,2.24,18,99,2.2,1.94,.3,1.46,2.62,1.23,3.16,450
-2,12.16,1.61,2.31,22.8,90,1.78,1.69,.43,1.56,2.45,1.33,2.26,495
-2,11.65,1.67,2.62,26,88,1.92,1.61,.4,1.34,2.6,1.36,3.21,562
-2,11.64,2.06,2.46,21.6,84,1.95,1.69,.48,1.35,2.8,1,2.75,680
-2,12.08,1.33,2.3,23.6,70,2.2,1.59,.42,1.38,1.74,1.07,3.21,625
-2,12.08,1.83,2.32,18.5,81,1.6,1.5,.52,1.64,2.4,1.08,2.27,480
-2,12,1.51,2.42,22,86,1.45,1.25,.5,1.63,3.6,1.05,2.65,450
-2,12.69,1.53,2.26,20.7,80,1.38,1.46,.58,1.62,3.05,.96,2.06,495
-2,12.29,2.83,2.22,18,88,2.45,2.25,.25,1.99,2.15,1.15,3.3,290
-2,11.62,1.99,2.28,18,98,3.02,2.26,.17,1.35,3.25,1.16,2.96,345
-2,12.47,1.52,2.2,19,162,2.5,2.27,.32,3.28,2.6,1.16,2.63,937
-2,11.81,2.12,2.74,21.5,134,1.6,.99,.14,1.56,2.5,.95,2.26,625
-2,12.29,1.41,1.98,16,85,2.55,2.5,.29,1.77,2.9,1.23,2.74,428
-2,12.37,1.07,2.1,18.5,88,3.52,3.75,.24,1.95,4.5,1.04,2.77,660
-2,12.29,3.17,2.21,18,88,2.85,2.99,.45,2.81,2.3,1.42,2.83,406
-2,12.08,2.08,1.7,17.5,97,2.23,2.17,.26,1.4,3.3,1.27,2.96,710
-2,12.6,1.34,1.9,18.5,88,1.45,1.36,.29,1.35,2.45,1.04,2.77,562
-2,12.34,2.45,2.46,21,98,2.56,2.11,.34,1.31,2.8,.8,3.38,438
-2,11.82,1.72,1.88,19.5,86,2.5,1.64,.37,1.42,2.06,.94,2.44,415
-2,12.51,1.73,1.98,20.5,85,2.2,1.92,.32,1.48,2.94,1.04,3.57,672
-2,12.42,2.55,2.27,22,90,1.68,1.84,.66,1.42,2.7,.86,3.3,315
-2,12.25,1.73,2.12,19,80,1.65,2.03,.37,1.63,3.4,1,3.17,510
-2,12.72,1.75,2.28,22.5,84,1.38,1.76,.48,1.63,3.3,.88,2.42,488
-2,12.22,1.29,1.94,19,92,2.36,2.04,.39,2.08,2.7,.86,3.02,312
-2,11.61,1.35,2.7,20,94,2.74,2.92,.29,2.49,2.65,.96,3.26,680
-2,11.46,3.74,1.82,19.5,107,3.18,2.58,.24,3.58,2.9,.75,2.81,562
-2,12.52,2.43,2.17,21,88,2.55,2.27,.26,1.22,2,.9,2.78,325
-2,11.76,2.68,2.92,20,103,1.75,2.03,.6,1.05,3.8,1.23,2.5,607
-2,11.41,.74,2.5,21,88,2.48,2.01,.42,1.44,3.08,1.1,2.31,434
-2,12.08,1.39,2.5,22.5,84,2.56,2.29,.43,1.04,2.9,.93,3.19,385
-2,11.03,1.51,2.2,21.5,85,2.46,2.17,.52,2.01,1.9,1.71,2.87,407
-2,11.82,1.47,1.99,20.8,86,1.98,1.6,.3,1.53,1.95,.95,3.33,495
-2,12.42,1.61,2.19,22.5,108,2,2.09,.34,1.61,2.06,1.06,2.96,345
-2,12.77,3.43,1.98,16,80,1.63,1.25,.43,.83,3.4,.7,2.12,372
-2,12,3.43,2,19,87,2,1.64,.37,1.87,1.28,.93,3.05,564
-2,11.45,2.4,2.42,20,96,2.9,2.79,.32,1.83,3.25,.8,3.39,625
-2,11.56,2.05,3.23,28.5,119,3.18,5.08,.47,1.87,6,.93,3.69,465
-2,12.42,4.43,2.73,26.5,102,2.2,2.13,.43,1.71,2.08,.92,3.12,365
-2,13.05,5.8,2.13,21.5,86,2.62,2.65,.3,2.01,2.6,.73,3.1,380
-2,11.87,4.31,2.39,21,82,2.86,3.03,.21,2.91,2.8,.75,3.64,380
-2,12.07,2.16,2.17,21,85,2.6,2.65,.37,1.35,2.76,.86,3.28,378
-2,12.43,1.53,2.29,21.5,86,2.74,3.15,.39,1.77,3.94,.69,2.84,352
-2,11.79,2.13,2.78,28.5,92,2.13,2.24,.58,1.76,3,.97,2.44,466
-2,12.37,1.63,2.3,24.5,88,2.22,2.45,.4,1.9,2.12,.89,2.78,342
-2,12.04,4.3,2.38,22,80,2.1,1.75,.42,1.35,2.6,.79,2.57,580
-3,12.86,1.35,2.32,18,122,1.51,1.25,.21,.94,4.1,.76,1.29,630
-3,12.88,2.99,2.4,20,104,1.3,1.22,.24,.83,5.4,.74,1.42,530
-3,12.81,2.31,2.4,24,98,1.15,1.09,.27,.83,5.7,.66,1.36,560
-3,12.7,3.55,2.36,21.5,106,1.7,1.2,.17,.84,5,.78,1.29,600
-3,12.51,1.24,2.25,17.5,85,2,.58,.6,1.25,5.45,.75,1.51,650
-3,12.6,2.46,2.2,18.5,94,1.62,.66,.63,.94,7.1,.73,1.58,695
-3,12.25,4.72,2.54,21,89,1.38,.47,.53,.8,3.85,.75,1.27,720
-3,12.53,5.51,2.64,25,96,1.79,.6,.63,1.1,5,.82,1.69,515
-3,13.49,3.59,2.19,19.5,88,1.62,.48,.58,.88,5.7,.81,1.82,580
-3,12.84,2.96,2.61,24,101,2.32,.6,.53,.81,4.92,.89,2.15,590
-3,12.93,2.81,2.7,21,96,1.54,.5,.53,.75,4.6,.77,2.31,600
-3,13.36,2.56,2.35,20,89,1.4,.5,.37,.64,5.6,.7,2.47,780
-3,13.52,3.17,2.72,23.5,97,1.55,.52,.5,.55,4.35,.89,2.06,520
-3,13.62,4.95,2.35,20,92,2,.8,.47,1.02,4.4,.91,2.05,550
-3,12.25,3.88,2.2,18.5,112,1.38,.78,.29,1.14,8.21,.65,2,855
-3,13.16,3.57,2.15,21,102,1.5,.55,.43,1.3,4,.6,1.68,830
-3,13.88,5.04,2.23,20,80,.98,.34,.4,.68,4.9,.58,1.33,415
-3,12.87,4.61,2.48,21.5,86,1.7,.65,.47,.86,7.65,.54,1.86,625
-3,13.32,3.24,2.38,21.5,92,1.93,.76,.45,1.25,8.42,.55,1.62,650
-3,13.08,3.9,2.36,21.5,113,1.41,1.39,.34,1.14,9.40,.57,1.33,550
-3,13.5,3.12,2.62,24,123,1.4,1.57,.22,1.25,8.60,.59,1.3,500
-3,12.79,2.67,2.48,22,112,1.48,1.36,.24,1.26,10.8,.48,1.47,480
-3,13.11,1.9,2.75,25.5,116,2.2,1.28,.26,1.56,7.1,.61,1.33,425
-3,13.23,3.3,2.28,18.5,98,1.8,.83,.61,1.87,10.52,.56,1.51,675
-3,12.58,1.29,2.1,20,103,1.48,.58,.53,1.4,7.6,.58,1.55,640
-3,13.17,5.19,2.32,22,93,1.74,.63,.61,1.55,7.9,.6,1.48,725
-3,13.84,4.12,2.38,19.5,89,1.8,.83,.48,1.56,9.01,.57,1.64,480
-3,12.45,3.03,2.64,27,97,1.9,.58,.63,1.14,7.5,.67,1.73,880
-3,14.34,1.68,2.7,25,98,2.8,1.31,.53,2.7,13,.57,1.96,660
-3,13.48,1.67,2.64,22.5,89,2.6,1.1,.52,2.29,11.75,.57,1.78,620
-3,12.36,3.83,2.38,21,88,2.3,.92,.5,1.04,7.65,.56,1.58,520
-3,13.69,3.26,2.54,20,107,1.83,.56,.5,.8,5.88,.96,1.82,680
-3,12.85,3.27,2.58,22,106,1.65,.6,.6,.96,5.58,.87,2.11,570
-3,12.96,3.45,2.35,18.5,106,1.39,.7,.4,.94,5.28,.68,1.75,675
-3,13.78,2.76,2.3,22,90,1.35,.68,.41,1.03,9.58,.7,1.68,615
-3,13.73,4.36,2.26,22.5,88,1.28,.47,.52,1.15,6.62,.78,1.75,520
-3,13.45,3.7,2.6,23,111,1.7,.92,.43,1.46,10.68,.85,1.56,695
-3,12.82,3.37,2.3,19.5,88,1.48,.66,.4,.97,10.26,.72,1.75,685
-3,13.58,2.58,2.69,24.5,105,1.55,.84,.39,1.54,8.66,.74,1.8,750
-3,13.4,4.6,2.86,25,112,1.98,.96,.27,1.11,8.5,.67,1.92,630
-3,12.2,3.03,2.32,19,96,1.25,.49,.4,.73,5.5,.66,1.83,510
-3,12.77,2.39,2.28,19.5,86,1.39,.51,.48,.64,9.899999,.57,1.63,470
-3,14.16,2.51,2.48,20,91,1.68,.7,.44,1.24,9.7,.62,1.71,660
-3,13.71,5.65,2.45,20.5,95,1.68,.61,.52,1.06,7.7,.64,1.74,740
-3,13.4,3.91,2.48,23,102,1.8,.75,.43,1.41,7.3,.7,1.56,750
-3,13.27,4.28,2.26,20,120,1.59,.69,.43,1.35,10.2,.59,1.56,835
-3,13.17,2.59,2.37,20,120,1.65,.68,.53,1.46,9.3,.6,1.62,840
-3,14.13,4.1,2.74,24.5,96,2.05,.76,.56,1.35,9.2,.61,1.6,560

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/resources/datasets/README.md
----------------------------------------------------------------------
diff --git a/examples/src/main/resources/datasets/README.md b/examples/src/main/resources/datasets/README.md
new file mode 100644
index 0000000..2f9c5ec
--- /dev/null
+++ b/examples/src/main/resources/datasets/README.md
@@ -0,0 +1,2 @@
+iris.txt and cleared_machines are from Lichman, M. (2013). UCI Machine Learning Repository [http://archive.ics.uci.edu/ml]. Irvine, CA: University of California, School of Information and Computer Science.
+Read more about machine dataset http://archive.ics.uci.edu/ml/machine-learning-databases/cpu-performance/machine.names
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/resources/datasets/boston_housing_dataset.txt
----------------------------------------------------------------------
diff --git a/examples/src/main/resources/datasets/boston_housing_dataset.txt b/examples/src/main/resources/datasets/boston_housing_dataset.txt
new file mode 100644
index 0000000..654a340
--- /dev/null
+++ b/examples/src/main/resources/datasets/boston_housing_dataset.txt
@@ -0,0 +1,505 @@
+0.02731,0.00,7.070,0,0.4690,6.4210,78.90,4.9671,2,242.0,17.80,396.90,9.14,21.60
+0.02729,0.00,7.070,0,0.4690,7.1850,61.10,4.9671,2,242.0,17.80,392.83,4.03,34.70
+0.03237,0.00,2.180,0,0.4580,6.9980,45.80,6.0622,3,222.0,18.70,394.63,2.94,33.40
+0.06905,0.00,2.180,0,0.4580,7.1470,54.20,6.0622,3,222.0,18.70,396.90,5.33,36.20
+0.02985,0.00,2.180,0,0.4580,6.4300,58.70,6.0622,3,222.0,18.70,394.12,5.21,28.70
+0.08829,12.50,7.870,0,0.5240,6.0120,66.60,5.5605,5,311.0,15.20,395.60,12.43,22.90
+0.14455,12.50,7.870,0,0.5240,6.1720,96.10,5.9505,5,311.0,15.20,396.90,19.15,27.10
+0.21124,12.50,7.870,0,0.5240,5.6310,100.00,6.0821,5,311.0,15.20,386.63,29.93,16.50
+0.17004,12.50,7.870,0,0.5240,6.0040,85.90,6.5921,5,311.0,15.20,386.71,17.10,18.90
+0.22489,12.50,7.870,0,0.5240,6.3770,94.30,6.3467,5,311.0,15.20,392.52,20.45,15.00
+0.11747,12.50,7.870,0,0.5240,6.0090,82.90,6.2267,5,311.0,15.20,396.90,13.27,18.90
+0.09378,12.50,7.870,0,0.5240,5.8890,39.00,5.4509,5,311.0,15.20,390.50,15.71,21.70
+0.62976,0.00,8.140,0,0.5380,5.9490,61.80,4.7075,4,307.0,21.00,396.90,8.26,20.40
+0.63796,0.00,8.140,0,0.5380,6.0960,84.50,4.4619,4,307.0,21.00,380.02,10.26,18.20
+0.62739,0.00,8.140,0,0.5380,5.8340,56.50,4.4986,4,307.0,21.00,395.62,8.47,19.90
+1.05393,0.00,8.140,0,0.5380,5.9350,29.30,4.4986,4,307.0,21.00,386.85,6.58,23.10
+0.78420,0.00,8.140,0,0.5380,5.9900,81.70,4.2579,4,307.0,21.00,386.75,14.67,17.50
+0.80271,0.00,8.140,0,0.5380,5.4560,36.60,3.7965,4,307.0,21.00,288.99,11.69,20.20
+0.72580,0.00,8.140,0,0.5380,5.7270,69.50,3.7965,4,307.0,21.00,390.95,11.28,18.20
+1.25179,0.00,8.140,0,0.5380,5.5700,98.10,3.7979,4,307.0,21.00,376.57,21.02,13.60
+0.85204,0.00,8.140,0,0.5380,5.9650,89.20,4.0123,4,307.0,21.00,392.53,13.83,19.60
+1.23247,0.00,8.140,0,0.5380,6.1420,91.70,3.9769,4,307.0,21.00,396.90,18.72,15.20
+0.98843,0.00,8.140,0,0.5380,5.8130,100.00,4.0952,4,307.0,21.00,394.54,19.88,14.50
+0.75026,0.00,8.140,0,0.5380,5.9240,94.10,4.3996,4,307.0,21.00,394.33,16.30,15.60
+0.84054,0.00,8.140,0,0.5380,5.5990,85.70,4.4546,4,307.0,21.00,303.42,16.51,13.90
+0.67191,0.00,8.140,0,0.5380,5.8130,90.30,4.6820,4,307.0,21.00,376.88,14.81,16.60
+0.95577,0.00,8.140,0,0.5380,6.0470,88.80,4.4534,4,307.0,21.00,306.38,17.28,14.80
+0.77299,0.00,8.140,0,0.5380,6.4950,94.40,4.4547,4,307.0,21.00,387.94,12.80,18.40
+1.00245,0.00,8.140,0,0.5380,6.6740,87.30,4.2390,4,307.0,21.00,380.23,11.98,21.00
+1.13081,0.00,8.140,0,0.5380,5.7130,94.10,4.2330,4,307.0,21.00,360.17,22.60,12.70
+1.35472,0.00,8.140,0,0.5380,6.0720,100.00,4.1750,4,307.0,21.00,376.73,13.04,14.50
+1.38799,0.00,8.140,0,0.5380,5.9500,82.00,3.9900,4,307.0,21.00,232.60,27.71,13.20
+1.15172,0.00,8.140,0,0.5380,5.7010,95.00,3.7872,4,307.0,21.00,358.77,18.35,13.10
+1.61282,0.00,8.140,0,0.5380,6.0960,96.90,3.7598,4,307.0,21.00,248.31,20.34,13.50
+0.06417,0.00,5.960,0,0.4990,5.9330,68.20,3.3603,5,279.0,19.20,396.90,9.68,18.90
+0.09744,0.00,5.960,0,0.4990,5.8410,61.40,3.3779,5,279.0,19.20,377.56,11.41,20.00
+0.08014,0.00,5.960,0,0.4990,5.8500,41.50,3.9342,5,279.0,19.20,396.90,8.77,21.00
+0.17505,0.00,5.960,0,0.4990,5.9660,30.20,3.8473,5,279.0,19.20,393.43,10.13,24.70
+0.02763,75.00,2.950,0,0.4280,6.5950,21.80,5.4011,3,252.0,18.30,395.63,4.32,30.80
+0.03359,75.00,2.950,0,0.4280,7.0240,15.80,5.4011,3,252.0,18.30,395.62,1.98,34.90
+0.12744,0.00,6.910,0,0.4480,6.7700,2.90,5.7209,3,233.0,17.90,385.41,4.84,26.60
+0.14150,0.00,6.910,0,0.4480,6.1690,6.60,5.7209,3,233.0,17.90,383.37,5.81,25.30
+0.15936,0.00,6.910,0,0.4480,6.2110,6.50,5.7209,3,233.0,17.90,394.46,7.44,24.70
+0.12269,0.00,6.910,0,0.4480,6.0690,40.00,5.7209,3,233.0,17.90,389.39,9.55,21.20
+0.17142,0.00,6.910,0,0.4480,5.6820,33.80,5.1004,3,233.0,17.90,396.90,10.21,19.30
+0.18836,0.00,6.910,0,0.4480,5.7860,33.30,5.1004,3,233.0,17.90,396.90,14.15,20.00
+0.22927,0.00,6.910,0,0.4480,6.0300,85.50,5.6894,3,233.0,17.90,392.74,18.80,16.60
+0.25387,0.00,6.910,0,0.4480,5.3990,95.30,5.8700,3,233.0,17.90,396.90,30.81,14.40
+0.21977,0.00,6.910,0,0.4480,5.6020,62.00,6.0877,3,233.0,17.90,396.90,16.20,19.40
+0.08873,21.00,5.640,0,0.4390,5.9630,45.70,6.8147,4,243.0,16.80,395.56,13.45,19.70
+0.04337,21.00,5.640,0,0.4390,6.1150,63.00,6.8147,4,243.0,16.80,393.97,9.43,20.50
+0.05360,21.00,5.640,0,0.4390,6.5110,21.10,6.8147,4,243.0,16.80,396.90,5.28,25.00
+0.04981,21.00,5.640,0,0.4390,5.9980,21.40,6.8147,4,243.0,16.80,396.90,8.43,23.40
+0.01360,75.00,4.000,0,0.4100,5.8880,47.60,7.3197,3,469.0,21.10,396.90,14.80,18.90
+0.01311,90.00,1.220,0,0.4030,7.2490,21.90,8.6966,5,226.0,17.90,395.93,4.81,35.40
+0.02055,85.00,0.740,0,0.4100,6.3830,35.70,9.1876,2,313.0,17.30,396.90,5.77,24.70
+0.01432,100.00,1.320,0,0.4110,6.8160,40.50,8.3248,5,256.0,15.10,392.90,3.95,31.60
+0.15445,25.00,5.130,0,0.4530,6.1450,29.20,7.8148,8,284.0,19.70,390.68,6.86,23.30
+0.10328,25.00,5.130,0,0.4530,5.9270,47.20,6.9320,8,284.0,19.70,396.90,9.22,19.60
+0.14932,25.00,5.130,0,0.4530,5.7410,66.20,7.2254,8,284.0,19.70,395.11,13.15,18.70
+0.17171,25.00,5.130,0,0.4530,5.9660,93.40,6.8185,8,284.0,19.70,378.08,14.44,16.00
+0.11027,25.00,5.130,0,0.4530,6.4560,67.80,7.2255,8,284.0,19.70,396.90,6.73,22.20
+0.12650,25.00,5.130,0,0.4530,6.7620,43.40,7.9809,8,284.0,19.70,395.58,9.50,25.00
+0.01951,17.50,1.380,0,0.4161,7.1040,59.50,9.2229,3,216.0,18.60,393.24,8.05,33.00
+0.03584,80.00,3.370,0,0.3980,6.2900,17.80,6.6115,4,337.0,16.10,396.90,4.67,23.50
+0.04379,80.00,3.370,0,0.3980,5.7870,31.10,6.6115,4,337.0,16.10,396.90,10.24,19.40
+0.05789,12.50,6.070,0,0.4090,5.8780,21.40,6.4980,4,345.0,18.90,396.21,8.10,22.00
+0.13554,12.50,6.070,0,0.4090,5.5940,36.80,6.4980,4,345.0,18.90,396.90,13.09,17.40
+0.12816,12.50,6.070,0,0.4090,5.8850,33.00,6.4980,4,345.0,18.90,396.90,8.79,20.90
+0.08826,0.00,10.810,0,0.4130,6.4170,6.60,5.2873,4,305.0,19.20,383.73,6.72,24.20
+0.15876,0.00,10.810,0,0.4130,5.9610,17.50,5.2873,4,305.0,19.20,376.94,9.88,21.70
+0.09164,0.00,10.810,0,0.4130,6.0650,7.80,5.2873,4,305.0,19.20,390.91,5.52,22.80
+0.19539,0.00,10.810,0,0.4130,6.2450,6.20,5.2873,4,305.0,19.20,377.17,7.54,23.40
+0.07896,0.00,12.830,0,0.4370,6.2730,6.00,4.2515,5,398.0,18.70,394.92,6.78,24.10
+0.09512,0.00,12.830,0,0.4370,6.2860,45.00,4.5026,5,398.0,18.70,383.23,8.94,21.40
+0.10153,0.00,12.830,0,0.4370,6.2790,74.50,4.0522,5,398.0,18.70,373.66,11.97,20.00
+0.08707,0.00,12.830,0,0.4370,6.1400,45.80,4.0905,5,398.0,18.70,386.96,10.27,20.80
+0.05646,0.00,12.830,0,0.4370,6.2320,53.70,5.0141,5,398.0,18.70,386.40,12.34,21.20
+0.08387,0.00,12.830,0,0.4370,5.8740,36.60,4.5026,5,398.0,18.70,396.06,9.10,20.30
+0.04113,25.00,4.860,0,0.4260,6.7270,33.50,5.4007,4,281.0,19.00,396.90,5.29,28.00
+0.04462,25.00,4.860,0,0.4260,6.6190,70.40,5.4007,4,281.0,19.00,395.63,7.22,23.90
+0.03659,25.00,4.860,0,0.4260,6.3020,32.20,5.4007,4,281.0,19.00,396.90,6.72,24.80
+0.03551,25.00,4.860,0,0.4260,6.1670,46.70,5.4007,4,281.0,19.00,390.64,7.51,22.90
+0.05059,0.00,4.490,0,0.4490,6.3890,48.00,4.7794,3,247.0,18.50,396.90,9.62,23.90
+0.05735,0.00,4.490,0,0.4490,6.6300,56.10,4.4377,3,247.0,18.50,392.30,6.53,26.60
+0.05188,0.00,4.490,0,0.4490,6.0150,45.10,4.4272,3,247.0,18.50,395.99,12.86,22.50
+0.07151,0.00,4.490,0,0.4490,6.1210,56.80,3.7476,3,247.0,18.50,395.15,8.44,22.20
+0.05660,0.00,3.410,0,0.4890,7.0070,86.30,3.4217,2,270.0,17.80,396.90,5.50,23.60
+0.05302,0.00,3.410,0,0.4890,7.0790,63.10,3.4145,2,270.0,17.80,396.06,5.70,28.70
+0.04684,0.00,3.410,0,0.4890,6.4170,66.10,3.0923,2,270.0,17.80,392.18,8.81,22.60
+0.03932,0.00,3.410,0,0.4890,6.4050,73.90,3.0921,2,270.0,17.80,393.55,8.20,22.00
+0.04203,28.00,15.040,0,0.4640,6.4420,53.60,3.6659,4,270.0,18.20,395.01,8.16,22.90
+0.02875,28.00,15.040,0,0.4640,6.2110,28.90,3.6659,4,270.0,18.20,396.33,6.21,25.00
+0.04294,28.00,15.040,0,0.4640,6.2490,77.30,3.6150,4,270.0,18.20,396.90,10.59,20.60
+0.12204,0.00,2.890,0,0.4450,6.6250,57.80,3.4952,2,276.0,18.00,357.98,6.65,28.40
+0.11504,0.00,2.890,0,0.4450,6.1630,69.60,3.4952,2,276.0,18.00,391.83,11.34,21.40
+0.12083,0.00,2.890,0,0.4450,8.0690,76.00,3.4952,2,276.0,18.00,396.90,4.21,38.70
+0.08187,0.00,2.890,0,0.4450,7.8200,36.90,3.4952,2,276.0,18.00,393.53,3.57,43.80
+0.06860,0.00,2.890,0,0.4450,7.4160,62.50,3.4952,2,276.0,18.00,396.90,6.19,33.20
+0.14866,0.00,8.560,0,0.5200,6.7270,79.90,2.7778,5,384.0,20.90,394.76,9.42,27.50
+0.11432,0.00,8.560,0,0.5200,6.7810,71.30,2.8561,5,384.0,20.90,395.58,7.67,26.50
+0.22876,0.00,8.560,0,0.5200,6.4050,85.40,2.7147,5,384.0,20.90,70.80,10.63,18.60
+0.21161,0.00,8.560,0,0.5200,6.1370,87.40,2.7147,5,384.0,20.90,394.47,13.44,19.30
+0.13960,0.00,8.560,0,0.5200,6.1670,90.00,2.4210,5,384.0,20.90,392.69,12.33,20.10
+0.13262,0.00,8.560,0,0.5200,5.8510,96.70,2.1069,5,384.0,20.90,394.05,16.47,19.50
+0.17120,0.00,8.560,0,0.5200,5.8360,91.90,2.2110,5,384.0,20.90,395.67,18.66,19.50
+0.13117,0.00,8.560,0,0.5200,6.1270,85.20,2.1224,5,384.0,20.90,387.69,14.09,20.40
+0.12802,0.00,8.560,0,0.5200,6.4740,97.10,2.4329,5,384.0,20.90,395.24,12.27,19.80
+0.26363,0.00,8.560,0,0.5200,6.2290,91.20,2.5451,5,384.0,20.90,391.23,15.55,19.40
+0.10793,0.00,8.560,0,0.5200,6.1950,54.40,2.7778,5,384.0,20.90,393.49,13.00,21.70
+0.10084,0.00,10.010,0,0.5470,6.7150,81.60,2.6775,6,432.0,17.80,395.59,10.16,22.80
+0.12329,0.00,10.010,0,0.5470,5.9130,92.90,2.3534,6,432.0,17.80,394.95,16.21,18.80
+0.22212,0.00,10.010,0,0.5470,6.0920,95.40,2.5480,6,432.0,17.80,396.90,17.09,18.70
+0.14231,0.00,10.010,0,0.5470,6.2540,84.20,2.2565,6,432.0,17.80,388.74,10.45,18.50
+0.17134,0.00,10.010,0,0.5470,5.9280,88.20,2.4631,6,432.0,17.80,344.91,15.76,18.30
+0.13158,0.00,10.010,0,0.5470,6.1760,72.50,2.7301,6,432.0,17.80,393.30,12.04,21.20
+0.15098,0.00,10.010,0,0.5470,6.0210,82.60,2.7474,6,432.0,17.80,394.51,10.30,19.20
+0.13058,0.00,10.010,0,0.5470,5.8720,73.10,2.4775,6,432.0,17.80,338.63,15.37,20.40
+0.14476,0.00,10.010,0,0.5470,5.7310,65.20,2.7592,6,432.0,17.80,391.50,13.61,19.30
+0.06899,0.00,25.650,0,0.5810,5.8700,69.70,2.2577,2,188.0,19.10,389.15,14.37,22.00
+0.07165,0.00,25.650,0,0.5810,6.0040,84.10,2.1974,2,188.0,19.10,377.67,14.27,20.30
+0.09299,0.00,25.650,0,0.5810,5.9610,92.90,2.0869,2,188.0,19.10,378.09,17.93,20.50
+0.15038,0.00,25.650,0,0.5810,5.8560,97.00,1.9444,2,188.0,19.10,370.31,25.41,17.30
+0.09849,0.00,25.650,0,0.5810,5.8790,95.80,2.0063,2,188.0,19.10,379.38,17.58,18.80
+0.16902,0.00,25.650,0,0.5810,5.9860,88.40,1.9929,2,188.0,19.10,385.02,14.81,21.40
+0.38735,0.00,25.650,0,0.5810,5.6130,95.60,1.7572,2,188.0,19.10,359.29,27.26,15.70
+0.25915,0.00,21.890,0,0.6240,5.6930,96.00,1.7883,4,437.0,21.20,392.11,17.19,16.20
+0.32543,0.00,21.890,0,0.6240,6.4310,98.80,1.8125,4,437.0,21.20,396.90,15.39,18.00
+0.88125,0.00,21.890,0,0.6240,5.6370,94.70,1.9799,4,437.0,21.20,396.90,18.34,14.30
+0.34006,0.00,21.890,0,0.6240,6.4580,98.90,2.1185,4,437.0,21.20,395.04,12.60,19.20
+1.19294,0.00,21.890,0,0.6240,6.3260,97.70,2.2710,4,437.0,21.20,396.90,12.26,19.60
+0.59005,0.00,21.890,0,0.6240,6.3720,97.90,2.3274,4,437.0,21.20,385.76,11.12,23.00
+0.32982,0.00,21.890,0,0.6240,5.8220,95.40,2.4699,4,437.0,21.20,388.69,15.03,18.40
+0.97617,0.00,21.890,0,0.6240,5.7570,98.40,2.3460,4,437.0,21.20,262.76,17.31,15.60
+0.55778,0.00,21.890,0,0.6240,6.3350,98.20,2.1107,4,437.0,21.20,394.67,16.96,18.10
+0.32264,0.00,21.890,0,0.6240,5.9420,93.50,1.9669,4,437.0,21.20,378.25,16.90,17.40
+0.35233,0.00,21.890,0,0.6240,6.4540,98.40,1.8498,4,437.0,21.20,394.08,14.59,17.10
+0.24980,0.00,21.890,0,0.6240,5.8570,98.20,1.6686,4,437.0,21.20,392.04,21.32,13.30
+0.54452,0.00,21.890,0,0.6240,6.1510,97.90,1.6687,4,437.0,21.20,396.90,18.46,17.80
+0.29090,0.00,21.890,0,0.6240,6.1740,93.60,1.6119,4,437.0,21.20,388.08,24.16,14.00
+1.62864,0.00,21.890,0,0.6240,5.0190,100.00,1.4394,4,437.0,21.20,396.90,34.41,14.40
+3.32105,0.00,19.580,1,0.8710,5.4030,100.00,1.3216,5,403.0,14.70,396.90,26.82,13.40
+4.09740,0.00,19.580,0,0.8710,5.4680,100.00,1.4118,5,403.0,14.70,396.90,26.42,15.60
+2.77974,0.00,19.580,0,0.8710,4.9030,97.80,1.3459,5,403.0,14.70,396.90,29.29,11.80
+2.37934,0.00,19.580,0,0.8710,6.1300,100.00,1.4191,5,403.0,14.70,172.91,27.80,13.80
+2.15505,0.00,19.580,0,0.8710,5.6280,100.00,1.5166,5,403.0,14.70,169.27,16.65,15.60
+2.36862,0.00,19.580,0,0.8710,4.9260,95.70,1.4608,5,403.0,14.70,391.71,29.53,14.60
+2.33099,0.00,19.580,0,0.8710,5.1860,93.80,1.5296,5,403.0,14.70,356.99,28.32,17.80
+2.73397,0.00,19.580,0,0.8710,5.5970,94.90,1.5257,5,403.0,14.70,351.85,21.45,15.40
+1.65660,0.00,19.580,0,0.8710,6.1220,97.30,1.6180,5,403.0,14.70,372.80,14.10,21.50
+1.49632,0.00,19.580,0,0.8710,5.4040,100.00,1.5916,5,403.0,14.70,341.60,13.28,19.60
+1.12658,0.00,19.580,1,0.8710,5.0120,88.00,1.6102,5,403.0,14.70,343.28,12.12,15.30
+2.14918,0.00,19.580,0,0.8710,5.7090,98.50,1.6232,5,403.0,14.70,261.95,15.79,19.40
+1.41385,0.00,19.580,1,0.8710,6.1290,96.00,1.7494,5,403.0,14.70,321.02,15.12,17.00
+3.53501,0.00,19.580,1,0.8710,6.1520,82.60,1.7455,5,403.0,14.70,88.01,15.02,15.60
+2.44668,0.00,19.580,0,0.8710,5.2720,94.00,1.7364,5,403.0,14.70,88.63,16.14,13.10
+1.22358,0.00,19.580,0,0.6050,6.9430,97.40,1.8773,5,403.0,14.70,363.43,4.59,41.30
+1.34284,0.00,19.580,0,0.6050,6.0660,100.00,1.7573,5,403.0,14.70,353.89,6.43,24.30
+1.42502,0.00,19.580,0,0.8710,6.5100,100.00,1.7659,5,403.0,14.70,364.31,7.39,23.30
+1.27346,0.00,19.580,1,0.6050,6.2500,92.60,1.7984,5,403.0,14.70,338.92,5.50,27.00
+1.46336,0.00,19.580,0,0.6050,7.4890,90.80,1.9709,5,403.0,14.70,374.43,1.73,50.00
+1.83377,0.00,19.580,1,0.6050,7.8020,98.20,2.0407,5,403.0,14.70,389.61,1.92,50.00
+1.51902,0.00,19.580,1,0.6050,8.3750,93.90,2.1620,5,403.0,14.70,388.45,3.32,50.00
+2.24236,0.00,19.580,0,0.6050,5.8540,91.80,2.4220,5,403.0,14.70,395.11,11.64,22.70
+2.92400,0.00,19.580,0,0.6050,6.1010,93.00,2.2834,5,403.0,14.70,240.16,9.81,25.00
+2.01019,0.00,19.580,0,0.6050,7.9290,96.20,2.0459,5,403.0,14.70,369.30,3.70,50.00
+1.80028,0.00,19.580,0,0.6050,5.8770,79.20,2.4259,5,403.0,14.70,227.61,12.14,23.80
+2.30040,0.00,19.580,0,0.6050,6.3190,96.10,2.1000,5,403.0,14.70,297.09,11.10,23.80
+2.44953,0.00,19.580,0,0.6050,6.4020,95.20,2.2625,5,403.0,14.70,330.04,11.32,22.30
+1.20742,0.00,19.580,0,0.6050,5.8750,94.60,2.4259,5,403.0,14.70,292.29,14.43,17.40
+2.31390,0.00,19.580,0,0.6050,5.8800,97.30,2.3887,5,403.0,14.70,348.13,12.03,19.10
+0.13914,0.00,4.050,0,0.5100,5.5720,88.50,2.5961,5,296.0,16.60,396.90,14.69,23.10
+0.09178,0.00,4.050,0,0.5100,6.4160,84.10,2.6463,5,296.0,16.60,395.50,9.04,23.60
+0.08447,0.00,4.050,0,0.5100,5.8590,68.70,2.7019,5,296.0,16.60,393.23,9.64,22.60
+0.06664,0.00,4.050,0,0.5100,6.5460,33.10,3.1323,5,296.0,16.60,390.96,5.33,29.40
+0.07022,0.00,4.050,0,0.5100,6.0200,47.20,3.5549,5,296.0,16.60,393.23,10.11,23.20
+0.05425,0.00,4.050,0,0.5100,6.3150,73.40,3.3175,5,296.0,16.60,395.60,6.29,24.60
+0.06642,0.00,4.050,0,0.5100,6.8600,74.40,2.9153,5,296.0,16.60,391.27,6.92,29.90
+0.05780,0.00,2.460,0,0.4880,6.9800,58.40,2.8290,3,193.0,17.80,396.90,5.04,37.20
+0.06588,0.00,2.460,0,0.4880,7.7650,83.30,2.7410,3,193.0,17.80,395.56,7.56,39.80
+0.06888,0.00,2.460,0,0.4880,6.1440,62.20,2.5979,3,193.0,17.80,396.90,9.45,36.20
+0.09103,0.00,2.460,0,0.4880,7.1550,92.20,2.7006,3,193.0,17.80,394.12,4.82,37.90
+0.10008,0.00,2.460,0,0.4880,6.5630,95.60,2.8470,3,193.0,17.80,396.90,5.68,32.50
+0.08308,0.00,2.460,0,0.4880,5.6040,89.80,2.9879,3,193.0,17.80,391.00,13.98,26.40
+0.06047,0.00,2.460,0,0.4880,6.1530,68.80,3.2797,3,193.0,17.80,387.11,13.15,29.60
+0.05602,0.00,2.460,0,0.4880,7.8310,53.60,3.1992,3,193.0,17.80,392.63,4.45,50.00
+0.07875,45.00,3.440,0,0.4370,6.7820,41.10,3.7886,5,398.0,15.20,393.87,6.68,32.00
+0.12579,45.00,3.440,0,0.4370,6.5560,29.10,4.5667,5,398.0,15.20,382.84,4.56,29.80
+0.08370,45.00,3.440,0,0.4370,7.1850,38.90,4.5667,5,398.0,15.20,396.90,5.39,34.90
+0.09068,45.00,3.440,0,0.4370,6.9510,21.50,6.4798,5,398.0,15.20,377.68,5.10,37.00
+0.06911,45.00,3.440,0,0.4370,6.7390,30.80,6.4798,5,398.0,15.20,389.71,4.69,30.50
+0.08664,45.00,3.440,0,0.4370,7.1780,26.30,6.4798,5,398.0,15.20,390.49,2.87,36.40
+0.02187,60.00,2.930,0,0.4010,6.8000,9.90,6.2196,1,265.0,15.60,393.37,5.03,31.10
+0.01439,60.00,2.930,0,0.4010,6.6040,18.80,6.2196,1,265.0,15.60,376.70,4.38,29.10
+0.01381,80.00,0.460,0,0.4220,7.8750,32.00,5.6484,4,255.0,14.40,394.23,2.97,50.00
+0.04011,80.00,1.520,0,0.4040,7.2870,34.10,7.3090,2,329.0,12.60,396.90,4.08,33.30
+0.04666,80.00,1.520,0,0.4040,7.1070,36.60,7.3090,2,329.0,12.60,354.31,8.61,30.30
+0.03768,80.00,1.520,0,0.4040,7.2740,38.30,7.3090,2,329.0,12.60,392.20,6.62,34.60
+0.03150,95.00,1.470,0,0.4030,6.9750,15.30,7.6534,3,402.0,17.00,396.90,4.56,34.90
+0.01778,95.00,1.470,0,0.4030,7.1350,13.90,7.6534,3,402.0,17.00,384.30,4.45,32.90
+0.03445,82.50,2.030,0,0.4150,6.1620,38.40,6.2700,2,348.0,14.70,393.77,7.43,24.10
+0.02177,82.50,2.030,0,0.4150,7.6100,15.70,6.2700,2,348.0,14.70,395.38,3.11,42.30
+0.03510,95.00,2.680,0,0.4161,7.8530,33.20,5.1180,4,224.0,14.70,392.78,3.81,48.50
+0.02009,95.00,2.680,0,0.4161,8.0340,31.90,5.1180,4,224.0,14.70,390.55,2.88,50.00
+0.13642,0.00,10.590,0,0.4890,5.8910,22.30,3.9454,4,277.0,18.60,396.90,10.87,22.60
+0.22969,0.00,10.590,0,0.4890,6.3260,52.50,4.3549,4,277.0,18.60,394.87,10.97,24.40
+0.25199,0.00,10.590,0,0.4890,5.7830,72.70,4.3549,4,277.0,18.60,389.43,18.06,22.50
+0.13587,0.00,10.590,1,0.4890,6.0640,59.10,4.2392,4,277.0,18.60,381.32,14.66,24.40
+0.43571,0.00,10.590,1,0.4890,5.3440,100.00,3.8750,4,277.0,18.60,396.90,23.09,20.00
+0.17446,0.00,10.590,1,0.4890,5.9600,92.10,3.8771,4,277.0,18.60,393.25,17.27,21.70
+0.37578,0.00,10.590,1,0.4890,5.4040,88.60,3.6650,4,277.0,18.60,395.24,23.98,19.30
+0.21719,0.00,10.590,1,0.4890,5.8070,53.80,3.6526,4,277.0,18.60,390.94,16.03,22.40
+0.14052,0.00,10.590,0,0.4890,6.3750,32.30,3.9454,4,277.0,18.60,385.81,9.38,28.10
+0.28955,0.00,10.590,0,0.4890,5.4120,9.80,3.5875,4,277.0,18.60,348.93,29.55,23.70
+0.19802,0.00,10.590,0,0.4890,6.1820,42.40,3.9454,4,277.0,18.60,393.63,9.47,25.00
+0.04560,0.00,13.890,1,0.5500,5.8880,56.00,3.1121,5,276.0,16.40,392.80,13.51,23.30
+0.07013,0.00,13.890,0,0.5500,6.6420,85.10,3.4211,5,276.0,16.40,392.78,9.69,28.70
+0.11069,0.00,13.890,1,0.5500,5.9510,93.80,2.8893,5,276.0,16.40,396.90,17.92,21.50
+0.11425,0.00,13.890,1,0.5500,6.3730,92.40,3.3633,5,276.0,16.40,393.74,10.50,23.00
+0.35809,0.00,6.200,1,0.5070,6.9510,88.50,2.8617,8,307.0,17.40,391.70,9.71,26.70
+0.40771,0.00,6.200,1,0.5070,6.1640,91.30,3.0480,8,307.0,17.40,395.24,21.46,21.70
+0.62356,0.00,6.200,1,0.5070,6.8790,77.70,3.2721,8,307.0,17.40,390.39,9.93,27.50
+0.61470,0.00,6.200,0,0.5070,6.6180,80.80,3.2721,8,307.0,17.40,396.90,7.60,30.10
+0.31533,0.00,6.200,0,0.5040,8.2660,78.30,2.8944,8,307.0,17.40,385.05,4.14,44.80
+0.52693,0.00,6.200,0,0.5040,8.7250,83.00,2.8944,8,307.0,17.40,382.00,4.63,50.00
+0.38214,0.00,6.200,0,0.5040,8.0400,86.50,3.2157,8,307.0,17.40,387.38,3.13,37.60
+0.41238,0.00,6.200,0,0.5040,7.1630,79.90,3.2157,8,307.0,17.40,372.08,6.36,31.60
+0.29819,0.00,6.200,0,0.5040,7.6860,17.00,3.3751,8,307.0,17.40,377.51,3.92,46.70
+0.44178,0.00,6.200,0,0.5040,6.5520,21.40,3.3751,8,307.0,17.40,380.34,3.76,31.50
+0.53700,0.00,6.200,0,0.5040,5.9810,68.10,3.6715,8,307.0,17.40,378.35,11.65,24.30
+0.46296,0.00,6.200,0,0.5040,7.4120,76.90,3.6715,8,307.0,17.40,376.14,5.25,31.70
+0.57529,0.00,6.200,0,0.5070,8.3370,73.30,3.8384,8,307.0,17.40,385.91,2.47,41.70
+0.33147,0.00,6.200,0,0.5070,8.2470,70.40,3.6519,8,307.0,17.40,378.95,3.95,48.30
+0.44791,0.00,6.200,1,0.5070,6.7260,66.50,3.6519,8,307.0,17.40,360.20,8.05,29.00
+0.33045,0.00,6.200,0,0.5070,6.0860,61.50,3.6519,8,307.0,17.40,376.75,10.88,24.00
+0.52058,0.00,6.200,1,0.5070,6.6310,76.50,4.1480,8,307.0,17.40,388.45,9.54,25.10
+0.51183,0.00,6.200,0,0.5070,7.3580,71.60,4.1480,8,307.0,17.40,390.07,4.73,31.50
+0.08244,30.00,4.930,0,0.4280,6.4810,18.50,6.1899,6,300.0,16.60,379.41,6.36,23.70
+0.09252,30.00,4.930,0,0.4280,6.6060,42.20,6.1899,6,300.0,16.60,383.78,7.37,23.30
+0.11329,30.00,4.930,0,0.4280,6.8970,54.30,6.3361,6,300.0,16.60,391.25,11.38,22.00
+0.10612,30.00,4.930,0,0.4280,6.0950,65.10,6.3361,6,300.0,16.60,394.62,12.40,20.10
+0.10290,30.00,4.930,0,0.4280,6.3580,52.90,7.0355,6,300.0,16.60,372.75,11.22,22.20
+0.12757,30.00,4.930,0,0.4280,6.3930,7.80,7.0355,6,300.0,16.60,374.71,5.19,23.70
+0.20608,22.00,5.860,0,0.4310,5.5930,76.50,7.9549,7,330.0,19.10,372.49,12.50,17.60
+0.19133,22.00,5.860,0,0.4310,5.6050,70.20,7.9549,7,330.0,19.10,389.13,18.46,18.50
+0.33983,22.00,5.860,0,0.4310,6.1080,34.90,8.0555,7,330.0,19.10,390.18,9.16,24.30
+0.19657,22.00,5.860,0,0.4310,6.2260,79.20,8.0555,7,330.0,19.10,376.14,10.15,20.50
+0.16439,22.00,5.860,0,0.4310,6.4330,49.10,7.8265,7,330.0,19.10,374.71,9.52,24.50
+0.19073,22.00,5.860,0,0.4310,6.7180,17.50,7.8265,7,330.0,19.10,393.74,6.56,26.20
+0.14030,22.00,5.860,0,0.4310,6.4870,13.00,7.3967,7,330.0,19.10,396.28,5.90,24.40
+0.21409,22.00,5.860,0,0.4310,6.4380,8.90,7.3967,7,330.0,19.10,377.07,3.59,24.80
+0.08221,22.00,5.860,0,0.4310,6.9570,6.80,8.9067,7,330.0,19.10,386.09,3.53,29.60
+0.36894,22.00,5.860,0,0.4310,8.2590,8.40,8.9067,7,330.0,19.10,396.90,3.54,42.80
+0.04819,80.00,3.640,0,0.3920,6.1080,32.00,9.2203,1,315.0,16.40,392.89,6.57,21.90
+0.03548,80.00,3.640,0,0.3920,5.8760,19.10,9.2203,1,315.0,16.40,395.18,9.25,20.90
+0.01538,90.00,3.750,0,0.3940,7.4540,34.20,6.3361,3,244.0,15.90,386.34,3.11,44.00
+0.61154,20.00,3.970,0,0.6470,8.7040,86.90,1.8010,5,264.0,13.00,389.70,5.12,50.00
+0.66351,20.00,3.970,0,0.6470,7.3330,100.00,1.8946,5,264.0,13.00,383.29,7.79,36.00
+0.65665,20.00,3.970,0,0.6470,6.8420,100.00,2.0107,5,264.0,13.00,391.93,6.90,30.10
+0.54011,20.00,3.970,0,0.6470,7.2030,81.80,2.1121,5,264.0,13.00,392.80,9.59,33.80
+0.53412,20.00,3.970,0,0.6470,7.5200,89.40,2.1398,5,264.0,13.00,388.37,7.26,43.10
+0.52014,20.00,3.970,0,0.6470,8.3980,91.50,2.2885,5,264.0,13.00,386.86,5.91,48.80
+0.82526,20.00,3.970,0,0.6470,7.3270,94.50,2.0788,5,264.0,13.00,393.42,11.25,31.00
+0.55007,20.00,3.970,0,0.6470,7.2060,91.60,1.9301,5,264.0,13.00,387.89,8.10,36.50
+0.76162,20.00,3.970,0,0.6470,5.5600,62.80,1.9865,5,264.0,13.00,392.40,10.45,22.80
+0.78570,20.00,3.970,0,0.6470,7.0140,84.60,2.1329,5,264.0,13.00,384.07,14.79,30.70
+0.57834,20.00,3.970,0,0.5750,8.2970,67.00,2.4216,5,264.0,13.00,384.54,7.44,50.00
+0.54050,20.00,3.970,0,0.5750,7.4700,52.60,2.8720,5,264.0,13.00,390.30,3.16,43.50
+0.09065,20.00,6.960,1,0.4640,5.9200,61.50,3.9175,3,223.0,18.60,391.34,13.65,20.70
+0.29916,20.00,6.960,0,0.4640,5.8560,42.10,4.4290,3,223.0,18.60,388.65,13.00,21.10
+0.16211,20.00,6.960,0,0.4640,6.2400,16.30,4.4290,3,223.0,18.60,396.90,6.59,25.20
+0.11460,20.00,6.960,0,0.4640,6.5380,58.70,3.9175,3,223.0,18.60,394.96,7.73,24.40
+0.22188,20.00,6.960,1,0.4640,7.6910,51.80,4.3665,3,223.0,18.60,390.77,6.58,35.20
+0.05644,40.00,6.410,1,0.4470,6.7580,32.90,4.0776,4,254.0,17.60,396.90,3.53,32.40
+0.09604,40.00,6.410,0,0.4470,6.8540,42.80,4.2673,4,254.0,17.60,396.90,2.98,32.00
+0.10469,40.00,6.410,1,0.4470,7.2670,49.00,4.7872,4,254.0,17.60,389.25,6.05,33.20
+0.06127,40.00,6.410,1,0.4470,6.8260,27.60,4.8628,4,254.0,17.60,393.45,4.16,33.10
+0.07978,40.00,6.410,0,0.4470,6.4820,32.10,4.1403,4,254.0,17.60,396.90,7.19,29.10
+0.21038,20.00,3.330,0,0.4429,6.8120,32.20,4.1007,5,216.0,14.90,396.90,4.85,35.10
+0.03578,20.00,3.330,0,0.4429,7.8200,64.50,4.6947,5,216.0,14.90,387.31,3.76,45.40
+0.03705,20.00,3.330,0,0.4429,6.9680,37.20,5.2447,5,216.0,14.90,392.23,4.59,35.40
+0.06129,20.00,3.330,1,0.4429,7.6450,49.70,5.2119,5,216.0,14.90,377.07,3.01,46.00
+0.01501,90.00,1.210,1,0.4010,7.9230,24.80,5.8850,1,198.0,13.60,395.52,3.16,50.00
+0.00906,90.00,2.970,0,0.4000,7.0880,20.80,7.3073,1,285.0,15.30,394.72,7.85,32.20
+0.01096,55.00,2.250,0,0.3890,6.4530,31.90,7.3073,1,300.0,15.30,394.72,8.23,22.00
+0.01965,80.00,1.760,0,0.3850,6.2300,31.50,9.0892,1,241.0,18.20,341.60,12.93,20.10
+0.03871,52.50,5.320,0,0.4050,6.2090,31.30,7.3172,6,293.0,16.60,396.90,7.14,23.20
+0.04590,52.50,5.320,0,0.4050,6.3150,45.60,7.3172,6,293.0,16.60,396.90,7.60,22.30
+0.04297,52.50,5.320,0,0.4050,6.5650,22.90,7.3172,6,293.0,16.60,371.72,9.51,24.80
+0.03502,80.00,4.950,0,0.4110,6.8610,27.90,5.1167,4,245.0,19.20,396.90,3.33,28.50
+0.07886,80.00,4.950,0,0.4110,7.1480,27.70,5.1167,4,245.0,19.20,396.90,3.56,37.30
+0.03615,80.00,4.950,0,0.4110,6.6300,23.40,5.1167,4,245.0,19.20,396.90,4.70,27.90
+0.08265,0.00,13.920,0,0.4370,6.1270,18.40,5.5027,4,289.0,16.00,396.90,8.58,23.90
+0.08199,0.00,13.920,0,0.4370,6.0090,42.30,5.5027,4,289.0,16.00,396.90,10.40,21.70
+0.12932,0.00,13.920,0,0.4370,6.6780,31.10,5.9604,4,289.0,16.00,396.90,6.27,28.60
+0.05372,0.00,13.920,0,0.4370,6.5490,51.00,5.9604,4,289.0,16.00,392.85,7.39,27.10
+0.14103,0.00,13.920,0,0.4370,5.7900,58.00,6.3200,4,289.0,16.00,396.90,15.84,20.30
+0.06466,70.00,2.240,0,0.4000,6.3450,20.10,7.8278,5,358.0,14.80,368.24,4.97,22.50
+0.05561,70.00,2.240,0,0.4000,7.0410,10.00,7.8278,5,358.0,14.80,371.58,4.74,29.00
+0.04417,70.00,2.240,0,0.4000,6.8710,47.40,7.8278,5,358.0,14.80,390.86,6.07,24.80
+0.03537,34.00,6.090,0,0.4330,6.5900,40.40,5.4917,7,329.0,16.10,395.75,9.50,22.00
+0.09266,34.00,6.090,0,0.4330,6.4950,18.40,5.4917,7,329.0,16.10,383.61,8.67,26.40
+0.10000,34.00,6.090,0,0.4330,6.9820,17.70,5.4917,7,329.0,16.10,390.43,4.86,33.10
+0.05515,33.00,2.180,0,0.4720,7.2360,41.10,4.0220,7,222.0,18.40,393.68,6.93,36.10
+0.05479,33.00,2.180,0,0.4720,6.6160,58.10,3.3700,7,222.0,18.40,393.36,8.93,28.40
+0.07503,33.00,2.180,0,0.4720,7.4200,71.90,3.0992,7,222.0,18.40,396.90,6.47,33.40
+0.04932,33.00,2.180,0,0.4720,6.8490,70.30,3.1827,7,222.0,18.40,396.90,7.53,28.20
+0.49298,0.00,9.900,0,0.5440,6.6350,82.50,3.3175,4,304.0,18.40,396.90,4.54,22.80
+0.34940,0.00,9.900,0,0.5440,5.9720,76.70,3.1025,4,304.0,18.40,396.24,9.97,20.30
+2.63548,0.00,9.900,0,0.5440,4.9730,37.80,2.5194,4,304.0,18.40,350.45,12.64,16.10
+0.79041,0.00,9.900,0,0.5440,6.1220,52.80,2.6403,4,304.0,18.40,396.90,5.98,22.10
+0.26169,0.00,9.900,0,0.5440,6.0230,90.40,2.8340,4,304.0,18.40,396.30,11.72,19.40
+0.26938,0.00,9.900,0,0.5440,6.2660,82.80,3.2628,4,304.0,18.40,393.39,7.90,21.60
+0.36920,0.00,9.900,0,0.5440,6.5670,87.30,3.6023,4,304.0,18.40,395.69,9.28,23.80
+0.25356,0.00,9.900,0,0.5440,5.7050,77.70,3.9450,4,304.0,18.40,396.42,11.50,16.20
+0.31827,0.00,9.900,0,0.5440,5.9140,83.20,3.9986,4,304.0,18.40,390.70,18.33,17.80
+0.24522,0.00,9.900,0,0.5440,5.7820,71.70,4.0317,4,304.0,18.40,396.90,15.94,19.80
+0.40202,0.00,9.900,0,0.5440,6.3820,67.20,3.5325,4,304.0,18.40,395.21,10.36,23.10
+0.47547,0.00,9.900,0,0.5440,6.1130,58.80,4.0019,4,304.0,18.40,396.23,12.73,21.00
+0.16760,0.00,7.380,0,0.4930,6.4260,52.30,4.5404,5,287.0,19.60,396.90,7.20,23.80
+0.18159,0.00,7.380,0,0.4930,6.3760,54.30,4.5404,5,287.0,19.60,396.90,6.87,23.10
+0.35114,0.00,7.380,0,0.4930,6.0410,49.90,4.7211,5,287.0,19.60,396.90,7.70,20.40
+0.28392,0.00,7.380,0,0.4930,5.7080,74.30,4.7211,5,287.0,19.60,391.13,11.74,18.50
+0.34109,0.00,7.380,0,0.4930,6.4150,40.10,4.7211,5,287.0,19.60,396.90,6.12,25.00
+0.19186,0.00,7.380,0,0.4930,6.4310,14.70,5.4159,5,287.0,19.60,393.68,5.08,24.60
+0.30347,0.00,7.380,0,0.4930,6.3120,28.90,5.4159,5,287.0,19.60,396.90,6.15,23.00
+0.24103,0.00,7.380,0,0.4930,6.0830,43.70,5.4159,5,287.0,19.60,396.90,12.79,22.20
+0.06617,0.00,3.240,0,0.4600,5.8680,25.80,5.2146,4,430.0,16.90,382.44,9.97,19.30
+0.06724,0.00,3.240,0,0.4600,6.3330,17.20,5.2146,4,430.0,16.90,375.21,7.34,22.60
+0.04544,0.00,3.240,0,0.4600,6.1440,32.20,5.8736,4,430.0,16.90,368.57,9.09,19.80
+0.05023,35.00,6.060,0,0.4379,5.7060,28.40,6.6407,1,304.0,16.90,394.02,12.43,17.10
+0.03466,35.00,6.060,0,0.4379,6.0310,23.30,6.6407,1,304.0,16.90,362.25,7.83,19.40
+0.05083,0.00,5.190,0,0.5150,6.3160,38.10,6.4584,5,224.0,20.20,389.71,5.68,22.20
+0.03738,0.00,5.190,0,0.5150,6.3100,38.50,6.4584,5,224.0,20.20,389.40,6.75,20.70
+0.03961,0.00,5.190,0,0.5150,6.0370,34.50,5.9853,5,224.0,20.20,396.90,8.01,21.10
+0.03427,0.00,5.190,0,0.5150,5.8690,46.30,5.2311,5,224.0,20.20,396.90,9.80,19.50
+0.03041,0.00,5.190,0,0.5150,5.8950,59.60,5.6150,5,224.0,20.20,394.81,10.56,18.50
+0.03306,0.00,5.190,0,0.5150,6.0590,37.30,4.8122,5,224.0,20.20,396.14,8.51,20.60
+0.05497,0.00,5.190,0,0.5150,5.9850,45.40,4.8122,5,224.0,20.20,396.90,9.74,19.00
+0.06151,0.00,5.190,0,0.5150,5.9680,58.50,4.8122,5,224.0,20.20,396.90,9.29,18.70
+0.01301,35.00,1.520,0,0.4420,7.2410,49.30,7.0379,1,284.0,15.50,394.74,5.49,32.70
+0.02498,0.00,1.890,0,0.5180,6.5400,59.70,6.2669,1,422.0,15.90,389.96,8.65,16.50
+0.02543,55.00,3.780,0,0.4840,6.6960,56.40,5.7321,5,370.0,17.60,396.90,7.18,23.90
+0.03049,55.00,3.780,0,0.4840,6.8740,28.10,6.4654,5,370.0,17.60,387.97,4.61,31.20
+0.03113,0.00,4.390,0,0.4420,6.0140,48.50,8.0136,3,352.0,18.80,385.64,10.53,17.50
+0.06162,0.00,4.390,0,0.4420,5.8980,52.30,8.0136,3,352.0,18.80,364.61,12.67,17.20
+0.01870,85.00,4.150,0,0.4290,6.5160,27.70,8.5353,4,351.0,17.90,392.43,6.36,23.10
+0.01501,80.00,2.010,0,0.4350,6.6350,29.70,8.3440,4,280.0,17.00,390.94,5.99,24.50
+0.02899,40.00,1.250,0,0.4290,6.9390,34.50,8.7921,1,335.0,19.70,389.85,5.89,26.60
+0.06211,40.00,1.250,0,0.4290,6.4900,44.40,8.7921,1,335.0,19.70,396.90,5.98,22.90
+0.07950,60.00,1.690,0,0.4110,6.5790,35.90,10.7103,4,411.0,18.30,370.78,5.49,24.10
+0.07244,60.00,1.690,0,0.4110,5.8840,18.50,10.7103,4,411.0,18.30,392.33,7.79,18.60
+0.01709,90.00,2.020,0,0.4100,6.7280,36.10,12.1265,5,187.0,17.00,384.46,4.50,30.10
+0.04301,80.00,1.910,0,0.4130,5.6630,21.90,10.5857,4,334.0,22.00,382.80,8.05,18.20
+0.10659,80.00,1.910,0,0.4130,5.9360,19.50,10.5857,4,334.0,22.00,376.04,5.57,20.60
+8.98296,0.00,18.100,1,0.7700,6.2120,97.40,2.1222,24,666.0,20.20,377.73,17.60,17.80
+3.84970,0.00,18.100,1,0.7700,6.3950,91.00,2.5052,24,666.0,20.20,391.34,13.27,21.70
+5.20177,0.00,18.100,1,0.7700,6.1270,83.40,2.7227,24,666.0,20.20,395.43,11.48,22.70
+4.26131,0.00,18.100,0,0.7700,6.1120,81.30,2.5091,24,666.0,20.20,390.74,12.67,22.60
+4.54192,0.00,18.100,0,0.7700,6.3980,88.00,2.5182,24,666.0,20.20,374.56,7.79,25.00
+3.83684,0.00,18.100,0,0.7700,6.2510,91.10,2.2955,24,666.0,20.20,350.65,14.19,19.90
+3.67822,0.00,18.100,0,0.7700,5.3620,96.20,2.1036,24,666.0,20.20,380.79,10.19,20.80
+4.22239,0.00,18.100,1,0.7700,5.8030,89.00,1.9047,24,666.0,20.20,353.04,14.64,16.80
+3.47428,0.00,18.100,1,0.7180,8.7800,82.90,1.9047,24,666.0,20.20,354.55,5.29,21.90
+4.55587,0.00,18.100,0,0.7180,3.5610,87.90,1.6132,24,666.0,20.20,354.70,7.12,27.50
+3.69695,0.00,18.100,0,0.7180,4.9630,91.40,1.7523,24,666.0,20.20,316.03,14.00,21.90
+13.52220,0.00,18.100,0,0.6310,3.8630,100.00,1.5106,24,666.0,20.20,131.42,13.33,23.10
+4.89822,0.00,18.100,0,0.6310,4.9700,100.00,1.3325,24,666.0,20.20,375.52,3.26,50.00
+5.66998,0.00,18.100,1,0.6310,6.6830,96.80,1.3567,24,666.0,20.20,375.33,3.73,50.00
+6.53876,0.00,18.100,1,0.6310,7.0160,97.50,1.2024,24,666.0,20.20,392.05,2.96,50.00
+9.23230,0.00,18.100,0,0.6310,6.2160,100.00,1.1691,24,666.0,20.20,366.15,9.53,50.00
+8.26725,0.00,18.100,1,0.6680,5.8750,89.60,1.1296,24,666.0,20.20,347.88,8.88,50.00
+11.10810,0.00,18.100,0,0.6680,4.9060,100.00,1.1742,24,666.0,20.20,396.90,34.77,13.80
+18.49820,0.00,18.100,0,0.6680,4.1380,100.00,1.1370,24,666.0,20.20,396.90,37.97,13.80
+19.60910,0.00,18.100,0,0.6710,7.3130,97.90,1.3163,24,666.0,20.20,396.90,13.44,15.00
+15.28800,0.00,18.100,0,0.6710,6.6490,93.30,1.3449,24,666.0,20.20,363.02,23.24,13.90
+9.82349,0.00,18.100,0,0.6710,6.7940,98.80,1.3580,24,666.0,20.20,396.90,21.24,13.30
+23.64820,0.00,18.100,0,0.6710,6.3800,96.20,1.3861,24,666.0,20.20,396.90,23.69,13.10
+17.86670,0.00,18.100,0,0.6710,6.2230,100.00,1.3861,24,666.0,20.20,393.74,21.78,10.20
+88.97620,0.00,18.100,0,0.6710,6.9680,91.90,1.4165,24,666.0,20.20,396.90,17.21,10.40
+15.87440,0.00,18.100,0,0.6710,6.5450,99.10,1.5192,24,666.0,20.20,396.90,21.08,10.90
+9.18702,0.00,18.100,0,0.7000,5.5360,100.00,1.5804,24,666.0,20.20,396.90,23.60,11.30
+7.99248,0.00,18.100,0,0.7000,5.5200,100.00,1.5331,24,666.0,20.20,396.90,24.56,12.30
+20.08490,0.00,18.100,0,0.7000,4.3680,91.20,1.4395,24,666.0,20.20,285.83,30.63,8.80
+16.81180,0.00,18.100,0,0.7000,5.2770,98.10,1.4261,24,666.0,20.20,396.90,30.81,7.20
+24.39380,0.00,18.100,0,0.7000,4.6520,100.00,1.4672,24,666.0,20.20,396.90,28.28,10.50
+22.59710,0.00,18.100,0,0.7000,5.0000,89.50,1.5184,24,666.0,20.20,396.90,31.99,7.40
+14.33370,0.00,18.100,0,0.7000,4.8800,100.00,1.5895,24,666.0,20.20,372.92,30.62,10.20
+8.15174,0.00,18.100,0,0.7000,5.3900,98.90,1.7281,24,666.0,20.20,396.90,20.85,11.50
+6.96215,0.00,18.100,0,0.7000,5.7130,97.00,1.9265,24,666.0,20.20,394.43,17.11,15.10
+5.29305,0.00,18.100,0,0.7000,6.0510,82.50,2.1678,24,666.0,20.20,378.38,18.76,23.20
+11.57790,0.00,18.100,0,0.7000,5.0360,97.00,1.7700,24,666.0,20.20,396.90,25.68,9.70
+8.64476,0.00,18.100,0,0.6930,6.1930,92.60,1.7912,24,666.0,20.20,396.90,15.17,13.80
+13.35980,0.00,18.100,0,0.6930,5.8870,94.70,1.7821,24,666.0,20.20,396.90,16.35,12.70
+8.71675,0.00,18.100,0,0.6930,6.4710,98.80,1.7257,24,666.0,20.20,391.98,17.12,13.10
+5.87205,0.00,18.100,0,0.6930,6.4050,96.00,1.6768,24,666.0,20.20,396.90,19.37,12.50
+7.67202,0.00,18.100,0,0.6930,5.7470,98.90,1.6334,24,666.0,20.20,393.10,19.92,8.50
+38.35180,0.00,18.100,0,0.6930,5.4530,100.00,1.4896,24,666.0,20.20,396.90,30.59,5.00
+9.91655,0.00,18.100,0,0.6930,5.8520,77.80,1.5004,24,666.0,20.20,338.16,29.97,6.30
+25.04610,0.00,18.100,0,0.6930,5.9870,100.00,1.5888,24,666.0,20.20,396.90,26.77,5.60
+14.23620,0.00,18.100,0,0.6930,6.3430,100.00,1.5741,24,666.0,20.20,396.90,20.32,7.20
+9.59571,0.00,18.100,0,0.6930,6.4040,100.00,1.6390,24,666.0,20.20,376.11,20.31,12.10
+24.80170,0.00,18.100,0,0.6930,5.3490,96.00,1.7028,24,666.0,20.20,396.90,19.77,8.30
+41.52920,0.00,18.100,0,0.6930,5.5310,85.40,1.6074,24,666.0,20.20,329.46,27.38,8.50
+67.92080,0.00,18.100,0,0.6930,5.6830,100.00,1.4254,24,666.0,20.20,384.97,22.98,5.00
+20.71620,0.00,18.100,0,0.6590,4.1380,100.00,1.1781,24,666.0,20.20,370.22,23.34,11.90
+11.95110,0.00,18.100,0,0.6590,5.6080,100.00,1.2852,24,666.0,20.20,332.09,12.13,27.90
+7.40389,0.00,18.100,0,0.5970,5.6170,97.90,1.4547,24,666.0,20.20,314.64,26.40,17.20
+14.43830,0.00,18.100,0,0.5970,6.8520,100.00,1.4655,24,666.0,20.20,179.36,19.78,27.50
+51.13580,0.00,18.100,0,0.5970,5.7570,100.00,1.4130,24,666.0,20.20,2.60,10.11,15.00
+14.05070,0.00,18.100,0,0.5970,6.6570,100.00,1.5275,24,666.0,20.20,35.05,21.22,17.20
+18.81100,0.00,18.100,0,0.5970,4.6280,100.00,1.5539,24,666.0,20.20,28.79,34.37,17.90
+28.65580,0.00,18.100,0,0.5970,5.1550,100.00,1.5894,24,666.0,20.20,210.97,20.08,16.30
+45.74610,0.00,18.100,0,0.6930,4.5190,100.00,1.6582,24,666.0,20.20,88.27,36.98,7.00
+18.08460,0.00,18.100,0,0.6790,6.4340,100.00,1.8347,24,666.0,20.20,27.25,29.05,7.20
+10.83420,0.00,18.100,0,0.6790,6.7820,90.80,1.8195,24,666.0,20.20,21.57,25.79,7.50
+25.94060,0.00,18.100,0,0.6790,5.3040,89.10,1.6475,24,666.0,20.20,127.36,26.64,10.40
+73.53410,0.00,18.100,0,0.6790,5.9570,100.00,1.8026,24,666.0,20.20,16.45,20.62,8.80
+11.81230,0.00,18.100,0,0.7180,6.8240,76.50,1.7940,24,666.0,20.20,48.45,22.74,8.40
+11.08740,0.00,18.100,0,0.7180,6.4110,100.00,1.8589,24,666.0,20.20,318.75,15.02,16.70
+7.02259,0.00,18.100,0,0.7180,6.0060,95.30,1.8746,24,666.0,20.20,319.98,15.70,14.20
+12.04820,0.00,18.100,0,0.6140,5.6480,87.60,1.9512,24,666.0,20.20,291.55,14.10,20.80
+7.05042,0.00,18.100,0,0.6140,6.1030,85.10,2.0218,24,666.0,20.20,2.52,23.29,13.40
+8.79212,0.00,18.100,0,0.5840,5.5650,70.60,2.0635,24,666.0,20.20,3.65,17.16,11.70
+15.86030,0.00,18.100,0,0.6790,5.8960,95.40,1.9096,24,666.0,20.20,7.68,24.39,8.30
+12.24720,0.00,18.100,0,0.5840,5.8370,59.70,1.9976,24,666.0,20.20,24.65,15.69,10.20
+37.66190,0.00,18.100,0,0.6790,6.2020,78.70,1.8629,24,666.0,20.20,18.82,14.52,10.90
+7.36711,0.00,18.100,0,0.6790,6.1930,78.10,1.9356,24,666.0,20.20,96.73,21.52,11.00
+9.33889,0.00,18.100,0,0.6790,6.3800,95.60,1.9682,24,666.0,20.20,60.72,24.08,9.50
+8.49213,0.00,18.100,0,0.5840,6.3480,86.10,2.0527,24,666.0,20.20,83.45,17.64,14.50
+10.06230,0.00,18.100,0,0.5840,6.8330,94.30,2.0882,24,666.0,20.20,81.33,19.69,14.10
+6.44405,0.00,18.100,0,0.5840,6.4250,74.80,2.2004,24,666.0,20.20,97.95,12.03,16.10
+5.58107,0.00,18.100,0,0.7130,6.4360,87.90,2.3158,24,666.0,20.20,100.19,16.22,14.30
+13.91340,0.00,18.100,0,0.7130,6.2080,95.00,2.2222,24,666.0,20.20,100.63,15.17,11.70
+11.16040,0.00,18.100,0,0.7400,6.6290,94.60,2.1247,24,666.0,20.20,109.85,23.27,13.40
+14.42080,0.00,18.100,0,0.7400,6.4610,93.30,2.0026,24,666.0,20.20,27.49,18.05,9.60
+15.17720,0.00,18.100,0,0.7400,6.1520,100.00,1.9142,24,666.0,20.20,9.32,26.45,8.70
+13.67810,0.00,18.100,0,0.7400,5.9350,87.90,1.8206,24,666.0,20.20,68.95,34.02,8.40
+9.39063,0.00,18.100,0,0.7400,5.6270,93.90,1.8172,24,666.0,20.20,396.90,22.88,12.80
+22.05110,0.00,18.100,0,0.7400,5.8180,92.40,1.8662,24,666.0,20.20,391.45,22.11,10.50
+9.72418,0.00,18.100,0,0.7400,6.4060,97.20,2.0651,24,666.0,20.20,385.96,19.52,17.10
+5.66637,0.00,18.100,0,0.7400,6.2190,100.00,2.0048,24,666.0,20.20,395.69,16.59,18.40
+9.96654,0.00,18.100,0,0.7400,6.4850,100.00,1.9784,24,666.0,20.20,386.73,18.85,15.40
+12.80230,0.00,18.100,0,0.7400,5.8540,96.60,1.8956,24,666.0,20.20,240.52,23.79,10.80
+10.67180,0.00,18.100,0,0.7400,6.4590,94.80,1.9879,24,666.0,20.20,43.06,23.98,11.80
+6.28807,0.00,18.100,0,0.7400,6.3410,96.40,2.0720,24,666.0,20.20,318.01,17.79,14.90
+9.92485,0.00,18.100,0,0.7400,6.2510,96.60,2.1980,24,666.0,20.20,388.52,16.44,12.60
+9.32909,0.00,18.100,0,0.7130,6.1850,98.70,2.2616,24,666.0,20.20,396.90,18.13,14.10
+7.52601,0.00,18.100,0,0.7130,6.4170,98.30,2.1850,24,666.0,20.20,304.21,19.31,13.00
+6.71772,0.00,18.100,0,0.7130,6.7490,92.60,2.3236,24,666.0,20.20,0.32,17.44,13.40
+5.44114,0.00,18.100,0,0.7130,6.6550,98.20,2.3552,24,666.0,20.20,355.29,17.73,15.20
+5.09017,0.00,18.100,0,0.7130,6.2970,91.80,2.3682,24,666.0,20.20,385.09,17.27,16.10
+8.24809,0.00,18.100,0,0.7130,7.3930,99.30,2.4527,24,666.0,20.20,375.87,16.74,17.80
+9.51363,0.00,18.100,0,0.7130,6.7280,94.10,2.4961,24,666.0,20.20,6.68,18.71,14.90
+4.75237,0.00,18.100,0,0.7130,6.5250,86.50,2.4358,24,666.0,20.20,50.92,18.13,14.10
+4.66883,0.00,18.100,0,0.7130,5.9760,87.90,2.5806,24,666.0,20.20,10.48,19.01,12.70
+8.20058,0.00,18.100,0,0.7130,5.9360,80.30,2.7792,24,666.0,20.20,3.50,16.94,13.50
+7.75223,0.00,18.100,0,0.7130,6.3010,83.70,2.7831,24,666.0,20.20,272.21,16.23,14.90
+6.80117,0.00,18.100,0,0.7130,6.0810,84.40,2.7175,24,666.0,20.20,396.90,14.70,20.00
+4.81213,0.00,18.100,0,0.7130,6.7010,90.00,2.5975,24,666.0,20.20,255.23,16.42,16.40
+3.69311,0.00,18.100,0,0.7130,6.3760,88.40,2.5671,24,666.0,20.20,391.43,14.65,17.70
+6.65492,0.00,18.100,0,0.7130,6.3170,83.00,2.7344,24,666.0,20.20,396.90,13.99,19.50
+5.82115,0.00,18.100,0,0.7130,6.5130,89.90,2.8016,24,666.0,20.20,393.82,10.29,20.20
+7.83932,0.00,18.100,0,0.6550,6.2090,65.40,2.9634,24,666.0,20.20,396.90,13.22,21.40
+3.16360,0.00,18.100,0,0.6550,5.7590,48.20,3.0665,24,666.0,20.20,334.40,14.13,19.90
+3.77498,0.00,18.100,0,0.6550,5.9520,84.70,2.8715,24,666.0,20.20,22.01,17.15,19.00
+4.42228,0.00,18.100,0,0.5840,6.0030,94.50,2.5403,24,666.0,20.20,331.29,21.32,19.10
+15.57570,0.00,18.100,0,0.5800,5.9260,71.00,2.9084,24,666.0,20.20,368.74,18.13,19.10
+13.07510,0.00,18.100,0,0.5800,5.7130,56.70,2.8237,24,666.0,20.20,396.90,14.76,20.10
+4.34879,0.00,18.100,0,0.5800,6.1670,84.00,3.0334,24,666.0,20.20,396.90,16.29,19.90
+4.03841,0.00,18.100,0,0.5320,6.2290,90.70,3.0993,24,666.0,20.20,395.33,12.87,19.60
+3.56868,0.00,18.100,0,0.5800,6.4370,75.00,2.8965,24,666.0,20.20,393.37,14.36,23.20
+4.64689,0.00,18.100,0,0.6140,6.9800,67.60,2.5329,24,666.0,20.20,374.68,11.66,29.80
+8.05579,0.00,18.100,0,0.5840,5.4270,95.40,2.4298,24,666.0,20.20,352.58,18.14,13.80
+6.39312,0.00,18.100,0,0.5840,6.1620,97.40,2.2060,24,666.0,20.20,302.76,24.10,13.30
+4.87141,0.00,18.100,0,0.6140,6.4840,93.60,2.3053,24,666.0,20.20,396.21,18.68,16.70
+15.02340,0.00,18.100,0,0.6140,5.3040,97.30,2.1007,24,666.0,20.20,349.48,24.91,12.00
+10.23300,0.00,18.100,0,0.6140,6.1850,96.70,2.1705,24,666.0,20.20,379.70,18.03,14.60
+14.33370,0.00,18.100,0,0.6140,6.2290,88.00,1.9512,24,666.0,20.20,383.32,13.11,21.40
+5.82401,0.00,18.100,0,0.5320,6.2420,64.70,3.4242,24,666.0,20.20,396.90,10.74,23.00
+5.70818,0.00,18.100,0,0.5320,6.7500,74.90,3.3317,24,666.0,20.20,393.07,7.74,23.70
+5.73116,0.00,18.100,0,0.5320,7.0610,77.00,3.4106,24,666.0,20.20,395.28,7.01,25.00
+2.81838,0.00,18.100,0,0.5320,5.7620,40.30,4.0983,24,666.0,20.20,392.92,10.42,21.80
+2.37857,0.00,18.100,0,0.5830,5.8710,41.90,3.7240,24,666.0,20.20,370.73,13.34,20.60
+3.67367,0.00,18.100,0,0.5830,6.3120,51.90,3.9917,24,666.0,20.20,388.62,10.58,21.20
+5.69175,0.00,18.100,0,0.5830,6.1140,79.80,3.5459,24,666.0,20.20,392.68,14.98,19.10
+4.83567,0.00,18.100,0,0.5830,5.9050,53.20,3.1523,24,666.0,20.20,388.22,11.45,20.60
+0.15086,0.00,27.740,0,0.6090,5.4540,92.70,1.8209,4,711.0,20.10,395.09,18.06,15.20
+0.18337,0.00,27.740,0,0.6090,5.4140,98.30,1.7554,4,711.0,20.10,344.05,23.97,7.00
+0.20746,0.00,27.740,0,0.6090,5.0930,98.00,1.8226,4,711.0,20.10,318.43,29.68,8.10
+0.10574,0.00,27.740,0,0.6090,5.9830,98.80,1.8681,4,711.0,20.10,390.11,18.07,13.60
+0.11132,0.00,27.740,0,0.6090,5.9830,83.50,2.1099,4,711.0,20.10,396.90,13.35,20.10
+0.17331,0.00,9.690,0,0.5850,5.7070,54.00,2.3817,6,391.0,19.20,396.90,12.01,21.80
+0.27957,0.00,9.690,0,0.5850,5.9260,42.60,2.3817,6,391.0,19.20,396.90,13.59,24.50
+0.17899,0.00,9.690,0,0.5850,5.6700,28.80,2.7986,6,391.0,19.20,393.29,17.60,23.10
+0.28960,0.00,9.690,0,0.5850,5.3900,72.90,2.7986,6,391.0,19.20,396.90,21.14,19.70
+0.26838,0.00,9.690,0,0.5850,5.7940,70.60,2.8927,6,391.0,19.20,396.90,14.10,18.30
+0.23912,0.00,9.690,0,0.5850,6.0190,65.30,2.4091,6,391.0,19.20,396.90,12.92,21.20
+0.17783,0.00,9.690,0,0.5850,5.5690,73.50,2.3999,6,391.0,19.20,395.77,15.10,17.50
+0.22438,0.00,9.690,0,0.5850,6.0270,79.70,2.4982,6,391.0,19.20,396.90,14.33,16.80
+0.06263,0.00,11.930,0,0.5730,6.5930,69.10,2.4786,1,273.0,21.00,391.99,9.67,22.40
+0.04527,0.00,11.930,0,0.5730,6.1200,76.70,2.2875,1,273.0,21.00,396.90,9.08,20.60
+0.06076,0.00,11.930,0,0.5730,6.9760,91.00,2.1675,1,273.0,21.00,396.90,5.64,23.90
+0.10959,0.00,11.930,0,0.5730,6.7940,89.30,2.3889,1,273.0,21.00,393.45,6.48,22.00
+0.04741,0.00,11.930,0,0.5730,6.0300,80.80,2.5050,1,273.0,21.00,396.90,7.88,11.90

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/resources/datasets/cleared_machines.csv
----------------------------------------------------------------------
diff --git a/examples/src/main/resources/datasets/cleared_machines.csv b/examples/src/main/resources/datasets/cleared_machines.csv
new file mode 100644
index 0000000..e22aac8
--- /dev/null
+++ b/examples/src/main/resources/datasets/cleared_machines.csv
@@ -0,0 +1,209 @@
+199;125;256;6000;256;16;128
+253;29;8000;32000;32;8;32
+253;29;8000;32000;32;8;32
+253;29;8000;32000;32;8;32
+132;29;8000;16000;32;8;16
+290;26;8000;32000;64;8;32
+381;23;16000;32000;64;16;32
+381;23;16000;32000;64;16;32
+749;23;16000;64000;64;16;32
+1238;23;32000;64000;128;32;64
+23;400;1000;3000;0;1;2
+24;400;512;3500;4;1;6
+70;60;2000;8000;65;1;8
+117;50;4000;16000;65;1;8
+15;350;64;64;0;1;4
+64;200;512;16000;0;4;32
+23;167;524;2000;8;4;15
+29;143;512;5000;0;7;32
+22;143;1000;2000;0;5;16
+124;110;5000;5000;142;8;64
+35;143;1500;6300;0;5;32
+39;143;3100;6200;0;5;20
+40;143;2300;6200;0;6;64
+45;110;3100;6200;0;6;64
+28;320;128;6000;0;1;12
+21;320;512;2000;4;1;3
+28;320;256;6000;0;1;6
+22;320;256;3000;4;1;3
+28;320;512;5000;4;1;5
+27;320;256;5000;4;1;6
+102;25;1310;2620;131;12;24
+102;25;1310;2620;131;12;24
+74;50;2620;10480;30;12;24
+74;50;2620;10480;30;12;24
+138;56;5240;20970;30;12;24
+136;64;5240;20970;30;12;24
+23;50;500;2000;8;1;4
+29;50;1000;4000;8;1;5
+44;50;2000;8000;8;1;5
+30;50;1000;4000;8;3;5
+41;50;1000;8000;8;3;5
+74;50;2000;16000;8;3;5
+74;50;2000;16000;8;3;6
+74;50;2000;16000;8;3;6
+54;133;1000;12000;9;3;12
+41;133;1000;8000;9;3;12
+18;810;512;512;8;1;1
+28;810;1000;5000;0;1;1
+36;320;512;8000;4;1;5
+38;200;512;8000;8;1;8
+34;700;384;8000;0;1;1
+19;700;256;2000;0;1;1
+72;140;1000;16000;16;1;3
+36;200;1000;8000;0;1;2
+30;110;1000;4000;16;1;2
+56;110;1000;12000;16;1;2
+42;220;1000;8000;16;1;2
+34;800;256;8000;0;1;4
+34;800;256;8000;0;1;4
+34;800;256;8000;0;1;4
+34;800;256;8000;0;1;4
+34;800;256;8000;0;1;4
+19;125;512;1000;0;8;20
+75;75;2000;8000;64;1;38
+113;75;2000;16000;64;1;38
+157;75;2000;16000;128;1;38
+18;90;256;1000;0;3;10
+20;105;256;2000;0;3;10
+28;105;1000;4000;0;3;24
+33;105;2000;4000;8;3;19
+47;75;2000;8000;8;3;24
+54;75;3000;8000;8;3;48
+20;175;256;2000;0;3;24
+23;300;768;3000;0;6;24
+25;300;768;3000;6;6;24
+52;300;768;12000;6;6;24
+27;300;768;4500;0;1;24
+50;300;384;12000;6;1;24
+18;300;192;768;6;6;24
+53;180;768;12000;6;1;31
+23;330;1000;3000;0;2;4
+30;300;1000;4000;8;3;64
+73;300;1000;16000;8;2;112
+20;330;1000;2000;0;1;2
+25;330;1000;4000;0;3;6
+28;140;2000;4000;0;3;6
+29;140;2000;4000;0;4;8
+32;140;2000;4000;8;1;20
+175;140;2000;32000;32;1;20
+57;140;2000;8000;32;1;54
+181;140;2000;32000;32;1;54
+181;140;2000;32000;32;1;54
+32;140;2000;4000;8;1;20
+82;57;4000;16000;1;6;12
+171;57;4000;24000;64;12;16
+361;26;16000;32000;64;16;24
+350;26;16000;32000;64;8;24
+220;26;8000;32000;0;8;24
+113;26;8000;16000;0;8;16
+15;480;96;512;0;1;1
+21;203;1000;2000;0;1;5
+35;115;512;6000;16;1;6
+18;1100;512;1500;0;1;1
+20;1100;768;2000;0;1;1
+20;600;768;2000;0;1;1
+28;400;2000;4000;0;1;1
+45;400;4000;8000;0;1;1
+18;900;1000;1000;0;1;2
+17;900;512;1000;0;1;2
+26;900;1000;4000;4;1;2
+28;900;1000;4000;8;1;2
+28;900;2000;4000;0;3;6
+31;225;2000;4000;8;3;6
+31;225;2000;4000;8;3;6
+42;180;2000;8000;8;1;6
+76;185;2000;16000;16;1;6
+76;180;2000;16000;16;1;6
+26;225;1000;4000;2;3;6
+59;25;2000;12000;8;1;4
+65;25;2000;12000;16;3;5
+101;17;4000;16000;8;6;12
+116;17;4000;16000;32;6;12
+18;1500;768;1000;0;0;0
+20;1500;768;2000;0;0;0
+20;800;768;2000;0;0;0
+30;50;2000;4000;0;3;6
+44;50;2000;8000;8;3;6
+44;50;2000;8000;8;1;6
+82;50;2000;16000;24;1;6
+82;50;2000;16000;24;1;6
+128;50;8000;16000;48;1;10
+37;100;1000;8000;0;2;6
+46;100;1000;8000;24;2;6
+46;100;1000;8000;24;3;6
+80;50;2000;16000;12;3;16
+88;50;2000;16000;24;6;16
+88;50;2000;16000;24;6;16
+33;150;512;4000;0;8;128
+46;115;2000;8000;16;1;3
+29;115;2000;4000;2;1;5
+53;92;2000;8000;32;1;6
+53;92;2000;8000;32;1;6
+41;92;2000;8000;4;1;6
+86;75;4000;16000;16;1;6
+95;60;4000;16000;32;1;6
+107;60;2000;16000;64;5;8
+117;60;4000;16000;64;5;8
+119;50;4000;16000;64;5;10
+120;72;4000;16000;64;8;16
+48;72;2000;8000;16;6;8
+126;40;8000;16000;32;8;16
+266;40;8000;32000;64;8;24
+270;35;8000;32000;64;8;24
+426;38;16000;32000;128;16;32
+151;48;4000;24000;32;8;24
+267;38;8000;32000;64;8;24
+603;30;16000;32000;256;16;24
+19;112;1000;1000;0;1;4
+21;84;1000;2000;0;1;6
+26;56;1000;4000;0;1;6
+35;56;2000;6000;0;1;8
+41;56;2000;8000;0;1;8
+47;56;4000;8000;0;1;8
+62;56;4000;12000;0;1;8
+78;56;4000;16000;0;1;8
+80;38;4000;8000;32;16;32
+80;38;4000;8000;32;16;32
+142;38;8000;16000;64;4;8
+281;38;8000;24000;160;4;8
+190;38;4000;16000;128;16;32
+21;200;1000;2000;0;1;2
+25;200;1000;4000;0;1;4
+67;200;2000;8000;64;1;5
+24;250;512;4000;0;1;7
+24;250;512;4000;0;4;7
+64;250;1000;16000;1;1;8
+25;160;512;4000;2;1;5
+20;160;512;2000;2;3;8
+29;160;1000;4000;8;1;14
+43;160;1000;8000;16;1;14
+53;160;2000;8000;32;1;13
+19;240;512;1000;8;1;3
+22;240;512;2000;8;1;5
+31;105;2000;4000;8;3;8
+41;105;2000;6000;16;6;16
+47;105;2000;8000;16;4;14
+99;52;4000;16000;32;4;12
+67;70;4000;12000;8;6;8
+81;59;4000;12000;32;6;12
+149;59;8000;16000;64;12;24
+183;26;8000;24000;32;8;16
+275;26;8000;32000;64;12;16
+382;26;8000;32000;128;24;32
+56;116;2000;8000;32;5;28
+182;50;2000;32000;24;6;26
+227;50;2000;32000;48;26;52
+341;50;2000;32000;112;52;104
+360;50;4000;32000;112;52;104
+919;30;8000;64000;96;12;176
+978;30;8000;64000;128;12;176
+24;180;262;4000;0;1;3
+24;180;512;4000;0;1;3
+24;180;262;4000;0;1;3
+24;180;512;4000;0;1;3
+37;124;1000;8000;0;1;8
+50;98;1000;8000;32;2;8
+41;125;2000;8000;0;2;14
+47;480;512;8000;32;0;0
+25;480;1000;4000;0;0;0

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/resources/datasets/glass_identification.csv
----------------------------------------------------------------------
diff --git a/examples/src/main/resources/datasets/glass_identification.csv b/examples/src/main/resources/datasets/glass_identification.csv
new file mode 100644
index 0000000..ae1d6d1
--- /dev/null
+++ b/examples/src/main/resources/datasets/glass_identification.csv
@@ -0,0 +1,116 @@
+1; 1.52101; 4.49; 1.10; 0.00; 0.00
+1; 1.51761; 3.60; 1.36; 0.00; 0.00
+1; 1.51618; 3.55; 1.54; 0.00; 0.00
+1; 1.51766; 3.69; 1.29; 0.00; 0.00
+1; 1.51742; 3.62; 1.24; 0.00; 0.00
+1; 1.51596; 3.61; 1.62; 0.00; 0.26
+1; 1.51743; 3.60; 1.14; 0.00; 0.00
+1; 1.51756; 3.61; 1.05; 0.00; 0.00
+1; 1.51918; 3.58; 1.37; 0.00; 0.00
+1; 1.51755; 3.60; 1.36; 0.00; 0.11
+1; 1.51571; 3.46; 1.56; 0.00; 0.24
+1; 1.51763; 3.66; 1.27; 0.00; 0.00
+1; 1.51589; 3.43; 1.40; 0.00; 0.24
+1; 1.51748; 3.56; 1.27; 0.00; 0.17
+1; 1.51763; 3.59; 1.31; 0.00; 0.00
+1; 1.51761; 3.54; 1.23; 0.00; 0.00
+1; 1.51784; 3.67; 1.16; 0.00; 0.00
+1; 1.52196; 3.85; 0.89; 0.00; 0.00
+1; 1.51911; 3.73; 1.18; 0.00; 0.00
+1; 1.51735; 3.54; 1.69; 0.00; 0.07
+1; 1.51750; 3.55; 1.49; 0.00; 0.19
+1; 1.51966; 3.75; 0.29; 0.00; 0.00
+1; 1.51736; 3.62; 1.29; 0.00; 0.00
+1; 1.51751; 3.57; 1.35; 0.00; 0.00
+1; 1.51720; 3.50; 1.15; 0.00; 0.00
+1; 1.51764; 3.54; 1.21; 0.00; 0.00
+1; 1.51793; 3.48; 1.41; 0.00; 0.00
+1; 1.51721; 3.48; 1.33; 0.00; 0.00
+1; 1.51768; 3.52; 1.43; 0.00; 0.00
+1; 1.51784; 3.49; 1.28; 0.00; 0.00
+1; 1.51768; 3.56; 1.30; 0.00; 0.14
+1; 1.51747; 3.50; 1.14; 0.00; 0.00
+1; 1.51775; 3.48; 1.23; 0.09; 0.22
+1; 1.51753; 3.47; 1.38; 0.00; 0.06
+1; 1.51783; 3.54; 1.34; 0.00; 0.00
+1; 1.51567; 3.45; 1.21; 0.00; 0.00
+1; 1.51909; 3.53; 1.32; 0.11; 0.00
+1; 1.51797; 3.48; 1.35; 0.00; 0.00
+1; 1.52213; 3.82; 0.47; 0.00; 0.00
+1; 1.52213; 3.82; 0.47; 0.00; 0.00
+1; 1.51793; 3.50; 1.12; 0.00; 0.00
+1; 1.51755; 3.42; 1.20; 0.00; 0.00
+1; 1.51779; 3.39; 1.33; 0.00; 0.00
+1; 1.52210; 3.84; 0.72; 0.00; 0.00
+1; 1.51786; 3.43; 1.19; 0.00; 0.30
+1; 1.51900; 3.48; 1.35; 0.00; 0.00
+1; 1.51869; 3.37; 1.18; 0.00; 0.16
+1; 1.52667; 3.70; 0.71; 0.00; 0.10
+1; 1.52223; 3.77; 0.79; 0.00; 0.00
+1; 1.51898; 3.35; 1.23; 0.00; 0.00
+1; 1.52320; 3.72; 0.51; 0.00; 0.16
+1; 1.51926; 3.33; 1.28; 0.00; 0.11
+1; 1.51808; 2.87; 1.19; 0.00; 0.00
+1; 1.51837; 2.84; 1.28; 0.00; 0.00
+1; 1.51778; 2.81; 1.29; 0.00; 0.09
+1; 1.51769; 2.71; 1.29; 0.00; 0.24
+1; 1.51215; 3.47; 1.12; 0.00; 0.31
+1; 1.51824; 3.48; 1.29; 0.00; 0.00
+1; 1.51754; 3.74; 1.17; 0.00; 0.00
+1; 1.51754; 3.66; 1.19; 0.00; 0.11
+1; 1.51905; 3.62; 1.11; 0.00; 0.00
+1; 1.51977; 3.58; 1.32; 0.69; 0.00
+1; 1.52172; 3.86; 0.88; 0.00; 0.11
+1; 1.52227; 3.81; 0.78; 0.00; 0.00
+1; 1.52172; 3.74; 0.90; 0.00; 0.07
+1; 1.52099; 3.59; 1.12; 0.00; 0.00
+1; 1.52152; 3.65; 0.87; 0.00; 0.17
+1; 1.52152; 3.65; 0.87; 0.00; 0.17
+1; 1.52152; 3.58; 0.90; 0.00; 0.16
+1; 1.52300; 3.58; 0.82; 0.00; 0.03
+3; 1.51769; 3.66; 1.11; 0.00; 0.00
+3; 1.51610; 3.53; 1.34; 0.00; 0.00
+3; 1.51670; 3.57; 1.38; 0.00; 0.10
+3; 1.51643; 3.52; 1.35; 0.00; 0.00
+3; 1.51665; 3.45; 1.76; 0.00; 0.17
+3; 1.52127; 3.90; 0.83; 0.00; 0.00
+3; 1.51779; 3.65; 0.65; 0.00; 0.00
+3; 1.51610; 3.40; 1.22; 0.00; 0.00
+3; 1.51694; 3.58; 1.31; 0.00; 0.00
+3; 1.51646; 3.40; 1.26; 0.00; 0.00
+3; 1.51655; 3.39; 1.28; 0.00; 0.00
+3; 1.52121; 3.76; 0.58; 0.00; 0.00
+3; 1.51776; 3.41; 1.52; 0.00; 0.00
+3; 1.51796; 3.36; 1.63; 0.00; 0.09
+3; 1.51832; 3.34; 1.54; 0.00; 0.00
+3; 1.51934; 3.54; 0.75; 0.15; 0.24
+3; 1.52211; 3.78; 0.91; 0.00; 0.37
+7; 1.51131; 3.20; 1.81; 1.19; 0.00
+7; 1.51838; 3.26; 2.22; 1.63; 0.00
+7; 1.52315; 3.34; 1.23; 0.00; 0.00
+7; 1.52247; 2.20; 2.06; 0.00; 0.00
+7; 1.52365; 1.83; 1.31; 1.68; 0.00
+7; 1.51613; 1.78; 1.79; 0.76; 0.00
+7; 1.51602; 0.00; 2.38; 0.64; 0.09
+7; 1.51623; 0.00; 2.79; 0.40; 0.09
+7; 1.51719; 0.00; 2.00; 1.59; 0.08
+7; 1.51683; 0.00; 1.98; 1.57; 0.07
+7; 1.51545; 0.00; 2.68; 0.61; 0.05
+7; 1.51556; 0.00; 2.54; 0.81; 0.01
+7; 1.51727; 0.00; 2.34; 0.66; 0.00
+7; 1.51531; 0.00; 2.66; 0.64; 0.00
+7; 1.51609; 0.00; 2.51; 0.53; 0.00
+7; 1.51508; 0.00; 2.25; 0.63; 0.00
+7; 1.51653; 0.00; 1.19; 0.00; 0.00
+7; 1.51514; 0.00; 2.42; 0.56; 0.00
+7; 1.51658; 0.00; 1.99; 1.71; 0.00
+7; 1.51617; 0.00; 2.27; 0.67; 0.00
+7; 1.51732; 0.00; 1.80; 1.55; 0.00
+7; 1.51645; 0.00; 1.87; 1.38; 0.00
+7; 1.51831; 0.00; 1.82; 2.88; 0.00
+7; 1.51640; 0.00; 2.74; 0.54; 0.00
+7; 1.51623; 0.00; 2.88; 1.06; 0.00
+7; 1.51685; 0.00; 1.99; 1.59; 0.00
+7; 1.52065; 0.00; 2.02; 1.64; 0.00
+7; 1.51651; 0.00; 1.94; 1.57; 0.00
+7; 1.51711; 0.00; 2.08; 1.67; 0.00

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/resources/datasets/iris.txt
----------------------------------------------------------------------
diff --git a/examples/src/main/resources/datasets/iris.txt b/examples/src/main/resources/datasets/iris.txt
new file mode 100644
index 0000000..18f5f7c
--- /dev/null
+++ b/examples/src/main/resources/datasets/iris.txt
@@ -0,0 +1,150 @@
+1.0	5.1	3.5	1.4	0.2
+1.0	4.9	3.0	1.4	0.2
+1.0	4.7	3.2	1.3	0.2
+1.0	4.6	3.1	1.5	0.2
+1.0	5.0	3.6	1.4	0.2
+1.0	5.4	3.9	1.7	0.4
+1.0	4.6	3.4	1.4	0.3
+1.0	5.0	3.4	1.5	0.2
+1.0	4.4	2.9	1.4	0.2
+1.0	4.9	3.1	1.5	0.1
+1.0	5.4	3.7	1.5	0.2
+1.0	4.8	3.4	1.6	0.2
+1.0	4.8	3.0	1.4	0.1
+1.0	4.3	3.0	1.1	0.1
+1.0	5.8	4.0	1.2	0.2
+1.0	5.7	4.4	1.5	0.4
+1.0	5.4	3.9	1.3	0.4
+1.0	5.1	3.5	1.4	0.3
+1.0	5.7	3.8	1.7	0.3
+1.0	5.1	3.8	1.5	0.3
+1.0	5.4	3.4	1.7	0.2
+1.0	5.1	3.7	1.5	0.4
+1.0	4.6	3.6	1.0	0.2
+1.0	5.1	3.3	1.7	0.5
+1.0	4.8	3.4	1.9	0.2
+1.0	5.0	3.0	1.6	0.2
+1.0	5.0	3.4	1.6	0.4
+1.0	5.2	3.5	1.5	0.2
+1.0	5.2	3.4	1.4	0.2
+1.0	4.7	3.2	1.6	0.2
+1.0	4.8	3.1	1.6	0.2
+1.0	5.4	3.4	1.5	0.4
+1.0	5.2	4.1	1.5	0.1
+1.0	5.5	4.2	1.4	0.2
+1.0	4.9	3.1	1.5	0.1
+1.0	5.0	3.2	1.2	0.2
+1.0	5.5	3.5	1.3	0.2
+1.0	4.9	3.1	1.5	0.1
+1.0	4.4	3.0	1.3	0.2
+1.0	5.1	3.4	1.5	0.2
+1.0	5.0	3.5	1.3	0.3
+1.0	4.5	2.3	1.3	0.3
+1.0	4.4	3.2	1.3	0.2
+1.0	5.0	3.5	1.6	0.6
+1.0	5.1	3.8	1.9	0.4
+1.0	4.8	3.0	1.4	0.3
+1.0	5.1	3.8	1.6	0.2
+1.0	4.6	3.2	1.4	0.2
+1.0	5.3	3.7	1.5	0.2
+1.0	5.0	3.3	1.4	0.2
+2.0	7.0	3.2	4.7	1.4
+2.0	6.4	3.2	4.5	1.5
+2.0	6.9	3.1	4.9	1.5
+2.0	5.5	2.3	4.0	1.3
+2.0	6.5	2.8	4.6	1.5
+2.0	5.7	2.8	4.5	1.3
+2.0	6.3	3.3	4.7	1.6
+2.0	4.9	2.4	3.3	1.0
+2.0	6.6	2.9	4.6	1.3
+2.0	5.2	2.7	3.9	1.4
+2.0	5.0	2.0	3.5	1.0
+2.0	5.9	3.0	4.2	1.5
+2.0	6.0	2.2	4.0	1.0
+2.0	6.1	2.9	4.7	1.4
+2.0	5.6	2.9	3.6	1.3
+2.0	6.7	3.1	4.4	1.4
+2.0	5.6	3.0	4.5	1.5
+2.0	5.8	2.7	4.1	1.0
+2.0	6.2	2.2	4.5	1.5
+2.0	5.6	2.5	3.9	1.1
+2.0	5.9	3.2	4.8	1.8
+2.0	6.1	2.8	4.0	1.3
+2.0	6.3	2.5	4.9	1.5
+2.0	6.1	2.8	4.7	1.2
+2.0	6.4	2.9	4.3	1.3
+2.0	6.6	3.0	4.4	1.4
+2.0	6.8	2.8	4.8	1.4
+2.0	6.7	3.0	5.0	1.7
+2.0	6.0	2.9	4.5	1.5
+2.0	5.7	2.6	3.5	1.0
+2.0	5.5	2.4	3.8	1.1
+2.0	5.5	2.4	3.7	1.0
+2.0	5.8	2.7	3.9	1.2
+2.0	6.0	2.7	5.1	1.6
+2.0	5.4	3.0	4.5	1.5
+2.0	6.0	3.4	4.5	1.6
+2.0	6.7	3.1	4.7	1.5
+2.0	6.3	2.3	4.4	1.3
+2.0	5.6	3.0	4.1	1.3
+2.0	5.5	2.5	4.0	1.3
+2.0	5.5	2.6	4.4	1.2
+2.0	6.1	3.0	4.6	1.4
+2.0	5.8	2.6	4.0	1.2
+2.0	5.0	2.3	3.3	1.0
+2.0	5.6	2.7	4.2	1.3
+2.0	5.7	3.0	4.2	1.2
+2.0	5.7	2.9	4.2	1.3
+2.0	6.2	2.9	4.3	1.3
+2.0	5.1	2.5	3.0	1.1
+2.0	5.7	2.8	4.1	1.3
+3.0	6.3	3.3	6.0	2.5
+3.0	5.8	2.7	5.1	1.9
+3.0	7.1	3.0	5.9	2.1
+3.0	6.3	2.9	5.6	1.8
+3.0	6.5	3.0	5.8	2.2
+3.0	7.6	3.0	6.6	2.1
+3.0	4.9	2.5	4.5	1.7
+3.0	7.3	2.9	6.3	1.8
+3.0	6.7	2.5	5.8	1.8
+3.0	7.2	3.6	6.1	2.5
+3.0	6.5	3.2	5.1	2.0
+3.0	6.4	2.7	5.3	1.9
+3.0	6.8	3.0	5.5	2.1
+3.0	5.7	2.5	5.0	2.0
+3.0	5.8	2.8	5.1	2.4
+3.0	6.4	3.2	5.3	2.3
+3.0	6.5	3.0	5.5	1.8
+3.0	7.7	3.8	6.7	2.2
+3.0	7.7	2.6	6.9	2.3
+3.0	6.0	2.2	5.0	1.5
+3.0	6.9	3.2	5.7	2.3
+3.0	5.6	2.8	4.9	2.0
+3.0	7.7	2.8	6.7	2.0
+3.0	6.3	2.7	4.9	1.8
+3.0	6.7	3.3	5.7	2.1
+3.0	7.2	3.2	6.0	1.8
+3.0	6.2	2.8	4.8	1.8
+3.0	6.1	3.0	4.9	1.8
+3.0	6.4	2.8	5.6	2.1
+3.0	7.2	3.0	5.8	1.6
+3.0	7.4	2.8	6.1	1.9
+3.0	7.9	3.8	6.4	2.0
+3.0	6.4	2.8	5.6	2.2
+3.0	6.3	2.8	5.1	1.5
+3.0	6.1	2.6	5.6	1.4
+3.0	7.7	3.0	6.1	2.3
+3.0	6.3	3.4	5.6	2.4
+3.0	6.4	3.1	5.5	1.8
+3.0	6.0	3.0	4.8	1.8
+3.0	6.9	3.1	5.4	2.1
+3.0	6.7	3.1	5.6	2.4
+3.0	6.9	3.1	5.1	2.3
+3.0	5.8	2.7	5.1	1.9
+3.0	6.8	3.2	5.9	2.3
+3.0	6.7	3.3	5.7	2.5
+3.0	6.7	3.0	5.2	2.3
+3.0	6.3	2.5	5.0	1.9
+3.0	6.5	3.0	5.2	2.0
+3.0	6.2	3.4	5.4	2.3
+3.0	5.9	3.0	5.1	1.8

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/resources/datasets/mortalitydata.csv
----------------------------------------------------------------------
diff --git a/examples/src/main/resources/datasets/mortalitydata.csv b/examples/src/main/resources/datasets/mortalitydata.csv
new file mode 100644
index 0000000..e4f3e41
--- /dev/null
+++ b/examples/src/main/resources/datasets/mortalitydata.csv
@@ -0,0 +1,53 @@
+8; 78; 284; 9.100000381; 109
+9.300000191; 68; 433; 8.699999809; 144
+7.5; 70; 739; 7.199999809; 113
+8.899999619; 96; 1792; 8.899999619; 97
+10.19999981; 74; 477; 8.300000191; 206
+8.300000191; 111; 362; 10.89999962; 124
+8.800000191; 77; 671; 10; 152
+8.800000191; 168; 636; 9.100000381; 162
+10.69999981; 82; 329; 8.699999809; 150
+11.69999981; 89; 634; 7.599999905; 134
+8.5; 149; 631; 10.80000019; 292
+8.300000191; 60; 257; 9.5; 108
+8.199999809; 96; 284; 8.800000191; 111
+7.900000095; 83; 603; 9.5; 182
+10.30000019; 130; 686; 8.699999809; 129
+7.400000095; 145; 345; 11.19999981; 158
+9.600000381; 112; 1357; 9.699999809; 186
+9.300000191; 131; 544; 9.600000381; 177
+10.60000038; 80; 205; 9.100000381; 127
+9.699999809; 130; 1264; 9.199999809; 179
+11.60000038; 140; 688; 8.300000191; 80
+8.100000381; 154; 354; 8.399999619; 103
+9.800000191; 118; 1632; 9.399999619; 101
+7.400000095; 94; 348; 9.800000191; 117
+9.399999619; 119; 370; 10.39999962; 88
+11.19999981; 153; 648; 9.899999619; 78
+9.100000381; 116; 366; 9.199999809; 102
+10.5; 97; 540; 10.30000019; 95
+11.89999962; 176; 680; 8.899999619; 80
+8.399999619; 75; 345; 9.600000381; 92
+5; 134; 525; 10.30000019; 126
+9.800000191; 161; 870; 10.39999962; 108
+9.800000191; 111; 669; 9.699999809; 77
+10.80000019; 114; 452; 9.600000381; 60
+10.10000038; 142; 430; 10.69999981; 71
+10.89999962; 238; 822; 10.30000019; 86
+9.199999809; 78; 190; 10.69999981; 93
+8.300000191; 196; 867; 9.600000381; 106
+7.300000191; 125; 969; 10.5; 162
+9.399999619; 82; 499; 7.699999809; 95
+9.399999619; 125; 925; 10.19999981; 91
+9.800000191; 129; 353; 9.899999619; 52
+3.599999905; 84; 288; 8.399999619; 110
+8.399999619; 183; 718; 10.39999962; 69
+10.80000019; 119; 540; 9.199999809; 57
+10.10000038; 180; 668; 13; 106
+9; 82; 347; 8.800000191; 40
+10; 71; 345; 9.199999809; 50
+11.30000019; 118; 463; 7.800000191; 35
+11.30000019; 121; 728; 8.199999809; 86
+12.80000019; 68; 383; 7.400000095; 57
+10; 112; 316; 10.39999962; 57
+6.699999809; 109; 388; 8.899999619; 94

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/resources/datasets/t10k-images-idx3-ubyte
----------------------------------------------------------------------
diff --git a/examples/src/main/resources/datasets/t10k-images-idx3-ubyte b/examples/src/main/resources/datasets/t10k-images-idx3-ubyte
new file mode 100644
index 0000000..261057a
Binary files /dev/null and b/examples/src/main/resources/datasets/t10k-images-idx3-ubyte differ

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/resources/datasets/t10k-labels-idx1-ubyte
----------------------------------------------------------------------
diff --git a/examples/src/main/resources/datasets/t10k-labels-idx1-ubyte b/examples/src/main/resources/datasets/t10k-labels-idx1-ubyte
new file mode 100644
index 0000000..d1c3a97
Binary files /dev/null and b/examples/src/main/resources/datasets/t10k-labels-idx1-ubyte differ


[09/50] [abbrv] ignite git commit: IGNITE-10339 Skip index partition file integrity check for in-memory caches - Fixes #5475.

Posted by ag...@apache.org.
IGNITE-10339 Skip index partition file integrity check for in-memory caches - Fixes #5475.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-9720
Commit: 5b656e2f3b2157200d650923e5aca800c4d61635
Parents: 97d2422
Author: Ivan Daschinskiy <iv...@gmail.com>
Authored: Fri Nov 23 19:36:00 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Nov 23 19:36:00 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/commandline/CommandHandler.java   | 6 ++++--
 .../ignite/internal/visor/verify/ValidateIndexesClosure.java | 8 +++++---
 2 files changed, 9 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5b656e2f/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java
index 56fbfae..e9a4281 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java
@@ -820,7 +820,11 @@ public class CommandHandler {
         VisorValidateIndexesTaskResult taskRes = executeTaskByNameOnNode(
             client, VALIDATE_INDEXES_TASK, taskArg, null);
 
+        boolean errors = false;
+
         if (!F.isEmpty(taskRes.exceptions())) {
+            errors = true;
+
             log("Index validation failed on nodes:");
 
             for (Map.Entry<UUID, Exception> e : taskRes.exceptions().entrySet()) {
@@ -832,8 +836,6 @@ public class CommandHandler {
             }
         }
 
-        boolean errors = false;
-
         for (Map.Entry<UUID, VisorValidateIndexesJobResult> nodeEntry : taskRes.results().entrySet()) {
             if (!nodeEntry.getValue().hasIssues())
                 continue;

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b656e2f/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java
index ec02c25..b6909e3 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java
@@ -260,7 +260,8 @@ public class ValidateIndexesClosure implements IgniteCallable<VisorValidateIndex
 
                 Map<PartitionKey, ValidateIndexesPartitionResult> partRes = fut.get();
 
-                partResults.putAll(partRes);
+                if (!partRes.isEmpty() && partRes.entrySet().stream().anyMatch(e -> !e.getValue().issues().isEmpty()))
+                    partResults.putAll(partRes);
             }
 
             for (; curIdx < procIdxFutures.size(); curIdx++) {
@@ -268,7 +269,8 @@ public class ValidateIndexesClosure implements IgniteCallable<VisorValidateIndex
 
                 Map<String, ValidateIndexesPartitionResult> idxRes = fut.get();
 
-                idxResults.putAll(idxRes);
+                if (!idxRes.isEmpty() && idxRes.entrySet().stream().anyMatch(e -> !e.getValue().issues().isEmpty()))
+                    idxResults.putAll(idxRes);
             }
 
             log.warning("ValidateIndexesClosure finished: processed " + totalPartitions + " partitions and "
@@ -296,7 +298,7 @@ public class ValidateIndexesClosure implements IgniteCallable<VisorValidateIndex
         for (Integer grpId: grpIds) {
             final CacheGroupContext grpCtx = ignite.context().cache().cacheGroup(grpId);
 
-            if (grpCtx == null) {
+            if (grpCtx == null || !grpCtx.persistenceEnabled()) {
                 integrityCheckedIndexes.incrementAndGet();
 
                 continue;


[30/50] [abbrv] ignite git commit: IGNITE-10002: MVCC: Create "Cache 2" test suite for MVCC mode. This closes #5198.

Posted by ag...@apache.org.
IGNITE-10002: MVCC: Create "Cache 2" test suite for MVCC mode. This closes #5198.


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

Branch: refs/heads/ignite-9720
Commit: 7577c8770eaf29e21f1d46f650af1d8fbd891c42
Parents: c63a60a
Author: Andrey V. Mashenkov <an...@gmail.com>
Authored: Tue Nov 27 11:45:12 2018 +0300
Committer: Igor Seliverstov <gv...@gmail.com>
Committed: Tue Nov 27 11:45:26 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |   3 +
 .../cache/mvcc/MvccProcessorImpl.java           |  11 +
 .../cache/CacheConcurrentReadThroughTest.java   |   8 +
 .../cache/CacheEnumOperationsAbstractTest.java  |  18 +-
 .../cache/CrossCacheTxRandomOperationsTest.java |  23 +-
 .../cache/GridCacheBasicApiAbstractTest.java    |  27 ++
 .../cache/GridCacheBasicStoreAbstractTest.java  |   8 +
 .../GridCacheEvictionEventAbstractTest.java     |   9 +
 .../GridCacheFinishPartitionsSelfTest.java      |   8 +
 ...idCacheGetAndTransformStoreAbstractTest.java |   8 +
 .../cache/GridCacheOffheapUpdateSelfTest.java   |  17 +-
 .../cache/GridCachePartitionedGetSelfTest.java  |   6 +
 .../GridCacheVariableTopologySelfTest.java      |   9 +
 .../IgniteCacheEntryProcessorNodeJoinTest.java  | 111 ++++---
 .../cache/IgniteCacheIncrementTxTest.java       |   7 +
 .../cache/IgniteCacheNoSyncForGetTest.java      |   8 +
 .../IgniteClientCacheStartFailoverTest.java     |  37 +++
 .../IgniteMvccTxMultiThreadedAbstractTest.java  | 121 +++++++
 .../IgniteMvccTxSingleThreadedAbstractTest.java |  50 +++
 .../cache/IgniteNearClientCacheCloseTest.java   |  22 ++
 .../processors/cache/IgniteTxAbstractTest.java  |  16 +-
 .../IgniteTxMultiThreadedAbstractTest.java      |   1 -
 ...CacheLoadingConcurrentGridStartSelfTest.java |   8 +
 .../CacheLockReleaseNodeLeaveTest.java          |   8 +
 .../GridCacheAbstractPrimarySyncSelfTest.java   |   4 +
 .../GridCacheBasicOpAbstractTest.java           |   8 +
 .../distributed/GridCacheLockAbstractTest.java  |   8 +
 .../GridCacheMultiNodeAbstractTest.java         |   5 +
 .../GridCacheMultiNodeLockAbstractTest.java     |   9 +
 .../GridCacheNodeFailureAbstractTest.java       |   8 +
 ...ridCachePartitionNotLoadedEventSelfTest.java |  13 +-
 ...NearDisabledMvccTxMultiThreadedSelfTest.java |  31 ++
 ...GridCachePreloadRestartAbstractSelfTest.java |  12 +
 .../GridCacheTransformEventSelfTest.java        |  64 +++-
 ...niteCacheClientNodeChangingTopologyTest.java |   8 +
 .../IgniteCacheTxIteratorSelfTest.java          |  27 +-
 .../IgniteMvccTxTimeoutAbstractTest.java        | 146 +++++++++
 .../IgniteTxTimeoutAbstractTest.java            |  39 ++-
 ...heColocatedMvccTxSingleThreadedSelfTest.java |  85 +++++
 .../dht/GridCacheDhtEntrySelfTest.java          |   5 +
 .../dht/GridCacheDhtMappingSelfTest.java        |   8 +
 .../GridCacheDhtPreloadDisabledSelfTest.java    |   8 +
 ...ridCacheDhtPreloadMultiThreadedSelfTest.java |   3 +
 .../dht/GridCacheDhtPreloadPutGetSelfTest.java  |   7 +
 .../dht/GridCacheDhtPreloadSelfTest.java        |   2 -
 .../IgniteCacheClearDuringRebalanceTest.java    |   9 +
 ...artitionedBackupNodeFailureRecoveryTest.java |   2 +-
 .../near/GridCacheNearClientHitTest.java        |   5 +-
 .../near/GridCacheNearMultiGetSelfTest.java     |  73 +++--
 .../near/GridCacheNearMultiNodeSelfTest.java    |  10 +
 .../near/GridCacheNearOneNodeSelfTest.java      |  11 +-
 .../GridCacheNearPartitionedClearSelfTest.java  |   8 +
 .../GridCacheNearReaderPreloadSelfTest.java     |   8 +
 .../near/GridCacheNearReadersSelfTest.java      |  10 +-
 ...ePartitionedBasicStoreMultiNodeSelfTest.java |   6 +
 .../near/GridCachePartitionedEventSelfTest.java |   8 +
 ...titionedExplicitLockNodeFailureSelfTest.java |   8 +
 .../GridCachePartitionedLoadCacheSelfTest.java  |   8 +
 ...ePartitionedMvccTxMultiThreadedSelfTest.java | 101 ++++++
 ...PartitionedMvccTxSingleThreadedSelfTest.java |  84 +++++
 ...idCachePartitionedMvccTxTimeoutSelfTest.java |  47 +++
 ...achePartitionedPreloadLifecycleSelfTest.java |   1 -
 .../GridCachePartitionedTxTimeoutSelfTest.java  |  21 --
 .../near/GridNearCacheStoreUpdateTest.java      |   9 +
 .../near/NearCacheSyncUpdateTest.java           |  17 +
 .../near/NoneRebalanceModeSelfTest.java         |   9 +
 .../GridCacheReplicatedTxTimeoutSelfTest.java   |  18 --
 .../local/GridCacheLocalBasicApiSelfTest.java   |   8 +
 .../local/GridCacheLocalBasicStoreSelfTest.java |   8 +
 .../local/GridCacheLocalEventSelfTest.java      |   9 +
 .../GridCacheLocalEvictionEventSelfTest.java    |   8 +
 ...dCacheLocalGetAndTransformStoreSelfTest.java |   8 +
 .../GridCacheLocalIsolatedNodesSelfTest.java    |   8 +
 .../local/GridCacheLocalLoadAllSelfTest.java    |   8 +
 .../cache/local/GridCacheLocalLockSelfTest.java |   8 +
 .../GridCacheLocalMultithreadedSelfTest.java    |   8 +
 .../GridCacheLocalTxMultiThreadedSelfTest.java  |   9 +
 .../GridCacheLocalTxSingleThreadedSelfTest.java |   8 +
 .../local/GridCacheLocalTxTimeoutSelfTest.java  |   8 +
 .../testframework/MvccFeatureChecker.java       | 137 ++++++++
 .../testsuites/IgniteCacheMvccTestSuite2.java   | 197 ++++++++++++
 .../testsuites/IgniteCacheTestSuite2.java       | 321 ++++++++++---------
 .../testsuites/IgniteCacheTestSuite6.java       |   3 -
 83 files changed, 1949 insertions(+), 324 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 2d27840..3c59ea1 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -1039,6 +1039,9 @@ public final class IgniteSystemProperties {
      */
     public static final String IGNITE_ALLOW_START_CACHES_IN_PARALLEL = "IGNITE_ALLOW_START_CACHES_IN_PARALLEL";
 
+    /** For test purposes only. Force Mvcc mode. */
+    public static final String IGNITE_FORCE_MVCC_MODE_IN_TESTS = "IGNITE_FORCE_MVCC_MODE_IN_TESTS";
+
     /**
      * Allows to log additional information about all restored partitions after binary and logical recovery phases.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
index c6515c2..3ea8f4c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
@@ -34,6 +34,7 @@ import java.util.stream.Collectors;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DataRegionConfiguration;
@@ -106,6 +107,7 @@ import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
@@ -134,6 +136,10 @@ import static org.apache.ignite.internal.processors.cache.persistence.CacheDataR
 @SuppressWarnings("serial")
 public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProcessor, DatabaseLifecycleListener {
     /** */
+    private static final boolean FORCE_MVCC =
+        IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_FORCE_MVCC_MODE_IN_TESTS, false);
+
+    /** */
     private static final IgniteProductVersion MVCC_SUPPORTED_SINCE = IgniteProductVersion.fromString("2.7.0");
 
     /** */
@@ -262,6 +268,11 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
 
     /** {@inheritDoc} */
     @Override public void preProcessCacheConfiguration(CacheConfiguration ccfg) {
+        if (FORCE_MVCC && ccfg.getAtomicityMode() == TRANSACTIONAL && !CU.isSystemCache(ccfg.getName())) {
+            ccfg.setAtomicityMode(TRANSACTIONAL_SNAPSHOT);
+            ccfg.setNearConfiguration(null);
+        }
+
         if (ccfg.getAtomicityMode() == TRANSACTIONAL_SNAPSHOT) {
             if (!mvccSupported)
                 throw new IgniteException("Cannot start MVCC transactional cache. " +

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConcurrentReadThroughTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConcurrentReadThroughTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConcurrentReadThroughTest.java
index 26f54a2..1e8ed4d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConcurrentReadThroughTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConcurrentReadThroughTest.java
@@ -34,6 +34,7 @@ import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 /**
@@ -50,6 +51,13 @@ public class CacheConcurrentReadThroughTest extends GridCommonAbstractTest {
     private boolean client;
 
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_STORE);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
index 148b60e..480d959 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
@@ -32,11 +32,15 @@ import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.junit.Assert.fail;
 
 /**
  *
@@ -92,7 +96,19 @@ public abstract class CacheEnumOperationsAbstractTest extends GridCommonAbstract
      * @throws Exception If failed.
      */
     public void testTx() throws Exception {
-        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC);
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, TRANSACTIONAL);
+
+        enumOperations(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTx() throws Exception {
+        if (!singleNode())
+            fail("https://issues.apache.org/jira/browse/IGNITE-7187");
+
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, TRANSACTIONAL_SNAPSHOT);
 
         enumOperations(ccfg);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
index cc9823b..cba3771 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
@@ -38,6 +38,7 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -92,6 +93,9 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
+        if (nearCacheEnabled())
+            MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
         super.beforeTestsStarted();
 
         startGridsMultiThreaded(GRID_CNT - 1);
@@ -194,6 +198,13 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
     private void txOperations(CacheMode cacheMode,
         CacheWriteSynchronizationMode writeSync,
         boolean crossCacheTx) throws Exception {
+        if (MvccFeatureChecker.forcedMvcc()) {
+            assert !nearCacheEnabled();
+
+            if(writeSync != CacheWriteSynchronizationMode.FULL_SYNC)
+                return;
+        }
+
         Ignite ignite = ignite(0);
 
         try {
@@ -203,12 +214,14 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
             txOperations(PESSIMISTIC, REPEATABLE_READ, crossCacheTx, false);
             txOperations(PESSIMISTIC, REPEATABLE_READ, crossCacheTx, true);
 
-            txOperations(OPTIMISTIC, REPEATABLE_READ, crossCacheTx, false);
-            txOperations(OPTIMISTIC, REPEATABLE_READ, crossCacheTx, true);
+            if(!MvccFeatureChecker.forcedMvcc()) {
+                txOperations(OPTIMISTIC, REPEATABLE_READ, crossCacheTx, false);
+                txOperations(OPTIMISTIC, REPEATABLE_READ, crossCacheTx, true);
 
-            if (writeSync == FULL_SYNC) {
-                txOperations(OPTIMISTIC, SERIALIZABLE, crossCacheTx, false);
-                txOperations(OPTIMISTIC, SERIALIZABLE, crossCacheTx, true);
+                if (writeSync == FULL_SYNC) {
+                    txOperations(OPTIMISTIC, SERIALIZABLE, crossCacheTx, false);
+                    txOperations(OPTIMISTIC, SERIALIZABLE, crossCacheTx, true);
+                }
             }
         }
         finally {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicApiAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicApiAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicApiAbstractTest.java
index f766d01..0b7de6e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicApiAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicApiAbstractTest.java
@@ -38,6 +38,7 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestThread;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
@@ -89,6 +90,8 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws Exception If test failed.
      */
     public void testBasicLock() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
         IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         Lock lock = cache.lock(1);
@@ -106,6 +109,8 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws IgniteCheckedException If test failed.
      */
     public void testSingleLockReentry() throws IgniteCheckedException {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
         IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         Lock lock = cache.lock(1);
@@ -134,6 +139,8 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws Exception If test failed.
      */
     public void testReentry() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
         IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         Lock lock = cache.lock(1);
@@ -173,6 +180,8 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      *
      */
     public void testInterruptLock() throws InterruptedException {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
         final IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         final Lock lock = cache.lock(1);
@@ -217,6 +226,8 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      *
      */
     public void testInterruptLockWithTimeout() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
         final IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         startGrid(1);
@@ -276,6 +287,8 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws IgniteCheckedException If test failed.
      */
     public void testManyLockReentries() throws IgniteCheckedException {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
         IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         Integer key = 1;
@@ -319,6 +332,8 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws Exception If test failed.
      */
     public void testLockMultithreaded() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
         final IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         final CountDownLatch l1 = new CountDownLatch(1);
@@ -437,6 +452,9 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws Exception If error occur.
      */
     public void testBasicOps() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_EVENTS);
+
         IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         CountDownLatch latch = new CountDownLatch(1);
@@ -498,6 +516,9 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws Exception If error occur.
      */
     public void testBasicOpsWithReentry() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_EVENTS);
+
         IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         int key = (int)System.currentTimeMillis();
@@ -570,6 +591,8 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws Exception If test failed.
      */
     public void testMultiLocks() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
         IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         Collection<Integer> keys = Arrays.asList(1, 2, 3);
@@ -625,6 +648,10 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws Exception In case of error.
      */
     public void testPutWithExpiration() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.EXPIRATION);
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_EVENTS);
+
         IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
         CacheEventListener lsnr = new CacheEventListener(new CountDownLatch(1));

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
index 1ae8be2..3ebb398 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.util.typedef.P2;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.jetbrains.annotations.Nullable;
@@ -53,6 +54,13 @@ public abstract class GridCacheBasicStoreAbstractTest extends GridCommonAbstract
     /** Cache store. */
     private static final GridCacheTestStore store = new GridCacheTestStore();
 
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_STORE);
+
+        super.setUp();
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEventAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEventAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEventAbstractTest.java
index 554a7a9..b2c87a9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEventAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEventAbstractTest.java
@@ -35,6 +35,7 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.events.EventType.EVT_CACHE_ENTRY_EVICTED;
@@ -49,6 +50,14 @@ public abstract class GridCacheEvictionEventAbstractTest extends GridCommonAbstr
     /** */
     private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.EVICTION);
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_EVENTS);
+
+        super.setUp();
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFinishPartitionsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFinishPartitionsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFinishPartitionsSelfTest.java
index 9732272..d60e4c6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFinishPartitionsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFinishPartitionsSelfTest.java
@@ -34,6 +34,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.transactions.Transaction;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -57,6 +58,13 @@ public class GridCacheFinishPartitionsSelfTest extends GridCacheAbstractSelfTest
     }
 
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         grid = (IgniteKernal)grid(0);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java
index f140945..af9751f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java
@@ -32,6 +32,7 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -48,6 +49,13 @@ public abstract class GridCacheGetAndTransformStoreAbstractTest extends GridComm
     /** Cache store. */
     private static final GridCacheTestStore store = new GridCacheTestStore();
 
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_STORE);
+
+        super.setUp();
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapUpdateSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapUpdateSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapUpdateSelfTest.java
index b8f6858..cf16417 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapUpdateSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapUpdateSelfTest.java
@@ -127,12 +127,19 @@ public class GridCacheOffheapUpdateSelfTest extends GridCommonAbstractTest {
 
             assertEquals(10, cache.get(key));
 
-            try (Transaction ignored = grid.transactions().txStart(OPTIMISTIC, REPEATABLE_READ)) {
-                assertEquals(10, cache.get(key));
+            if(((IgniteCacheProxy)cache).context().config().getAtomicityMode() != CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT) {
+                try (Transaction ignored = grid.transactions().txStart(OPTIMISTIC, REPEATABLE_READ)) {
+                    assertEquals(10, cache.get(key));
+                }
+
+                try (Transaction ignored = grid.transactions().txStart(PESSIMISTIC, READ_COMMITTED)) {
+                    assertEquals(10, cache.get(key));
+                }
             }
-
-            try (Transaction ignored = grid.transactions().txStart(PESSIMISTIC, READ_COMMITTED)) {
-                assertEquals(10, cache.get(key));
+            else {
+                try (Transaction ignored = grid.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                    assertEquals(10, cache.get(key));
+                }
             }
         }
         finally {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java
index 1ed5a12..f4c32ee 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java
@@ -33,6 +33,7 @@ import org.apache.ignite.spi.discovery.DiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
@@ -129,6 +130,11 @@ public class GridCachePartitionedGetSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testGetFromBackupNode() throws Exception {
+        if (MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-10274");
+
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.EVICTION);
+
         for (int i = 0; i < GRID_CNT; i++) {
             IgniteCache<String, Integer> c = grid(i).cache(DEFAULT_CACHE_NAME);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java
index 25817a1..b9f7a35 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java
@@ -35,6 +35,7 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionRollbackException;
@@ -52,6 +53,14 @@ public class GridCacheVariableTopologySelfTest extends GridCommonAbstractTest {
     /** */
     private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        if (MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-7388");
+
+        super.setUp();
+    }
+
     /** Constructs test. */
     public GridCacheVariableTopologySelfTest() {
         super(/* don't start grid */ false);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
index d6afb49..5f99b55 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorNodeJoinTest.java
@@ -43,9 +43,11 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.GridTestUtils.SF;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
@@ -115,7 +117,7 @@ public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTes
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        startGrids(GRID_CNT);
+        startGridsMultiThreaded(GRID_CNT, true);
     }
 
     /** {@inheritDoc} */
@@ -141,10 +143,13 @@ public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTes
      * @throws Exception If failed.
      */
     public void testEntryProcessorNodeLeave() throws Exception {
+        if (MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-10254");
+
         startGrid(GRID_CNT);
 
         // TODO: IGNITE-1525 (test fails with one-phase commit).
-        boolean createCache = atomicityMode() == TRANSACTIONAL;
+        boolean createCache = atomicityMode() != ATOMIC;
 
         String cacheName = DEFAULT_CACHE_NAME;
 
@@ -212,32 +217,34 @@ public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTes
      * @throws Exception If failed.
      */
     private void checkEntryProcessorNodeJoin(boolean invokeAll) throws Exception {
+        if (MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-10391");
+
         final AtomicBoolean stop = new AtomicBoolean();
         final AtomicReference<Throwable> error = new AtomicReference<>();
         final int started = 6;
 
-        try {
-            IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
-                @Override public void run() {
-                    try {
-                        for (int i = 0; i < started; i++) {
-                            U.sleep(1_000);
+        IgniteInternalFuture fut = GridTestUtils.runAsync(new Runnable() {
+            @Override public void run() {
+                try {
+                    for (int i = 0; i < started && !stop.get(); i++) {
+                        U.sleep(1_000);
 
+                        if (!stop.get())
                             startGrid(GRID_CNT + i);
-                        }
-                    }
-                    catch (Exception e) {
-                        error.compareAndSet(null, e);
                     }
                 }
-            }, 1, "starter");
+                catch (Exception e) {
+                    error.compareAndSet(null, e);
+                }
+            }
+        }, "starter");
 
+        try {
             try {
                 checkIncrement(DEFAULT_CACHE_NAME, invokeAll, null, null);
             }
             finally {
-                stop.set(true);
-
                 fut.get(getTestTimeout());
             }
 
@@ -251,8 +258,10 @@ public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTes
             }
         }
         finally {
-            for (int i = 0; i < started; i++)
-                stopGrid(GRID_CNT + i);
+            stop.set(true);
+
+            if (!fut.isDone())
+                fut.cancel();
         }
     }
 
@@ -331,51 +340,51 @@ public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTes
         final AtomicReference<Throwable> error = new AtomicReference<>();
         final int started = 6;
 
-        try {
-            int keys = 100;
+        int keys = 100;
 
-            final AtomicBoolean done = new AtomicBoolean(false);
+        final AtomicBoolean stop = new AtomicBoolean(false);
 
-            for (int i = 0; i < keys; i++)
-                ignite(0).cache(DEFAULT_CACHE_NAME).put(i, 0);
+        for (int i = 0; i < keys; i++)
+            ignite(0).cache(DEFAULT_CACHE_NAME).put(i, 0);
 
-            IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
-                @Override public void run() {
-                    try {
-                        for (int i = 0; i < started; i++) {
-                            U.sleep(1_000);
+        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+            @Override public void run() {
+                try {
+                    for (int i = 0; i < started && !stop.get(); i++) {
+                        U.sleep(1_000);
 
-                            IgniteEx grid = startGrid(GRID_CNT + i);
+                        if (stop.get())
+                            continue;
 
-                            info("Test started grid [idx=" + (GRID_CNT + i) + ", nodeId=" + grid.localNode().id() + ']');
-                        }
-                    }
-                    catch (Exception e) {
-                        error.compareAndSet(null, e);
-                    }
-                    finally {
-                        done.set(true);
+                        IgniteEx grid = startGrid(GRID_CNT + i);
+
+                        info("Test started grid [idx=" + (GRID_CNT + i) + ", nodeId=" + grid.localNode().id() + ']');
                     }
                 }
-            }, 1, "starter");
+                catch (Exception e) {
+                    error.compareAndSet(null, e);
+                }
+                finally {
+                    stop.set(true);
+                }
+            }
+        }, 1, "starter");
 
+        try {
             int updVal = 0;
 
-            try {
-                while (!done.get()) {
-                    info("Will put: " + (updVal + 1));
+            while (!stop.get()) {
+                info("Will put: " + (updVal + 1));
 
-                    for (int i = 0; i < keys; i++)
-                        assertTrue("Failed [key=" + i + ", oldVal=" + updVal + ']',
-                            ignite(0).cache(DEFAULT_CACHE_NAME).replace(i, updVal, updVal + 1));
+                for (int i = 0; i < keys; i++)
+                    assertTrue("Failed [key=" + i + ", oldVal=" + updVal + ']',
+                        ignite(0).cache(DEFAULT_CACHE_NAME).replace(i, updVal, updVal + 1));
 
-                    updVal++;
-                }
-            }
-            finally {
-                fut.get(getTestTimeout());
+                updVal++;
             }
 
+            fut.get(getTestTimeout());
+
             for (int i = 0; i < keys; i++) {
                 for (int g = 0; g < GRID_CNT + started; g++) {
                     Integer val = ignite(g).<Integer, Integer>cache(DEFAULT_CACHE_NAME).get(i);
@@ -390,8 +399,10 @@ public class IgniteCacheEntryProcessorNodeJoinTest extends GridCommonAbstractTes
             }
         }
         finally {
-            for (int i = 0; i < started; i++)
-                stopGrid(GRID_CNT + i);
+            stop.set(true);
+
+            if (!fut.isDone())
+                fut.cancel();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheIncrementTxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheIncrementTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheIncrementTxTest.java
index b3a5055..918d655 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheIncrementTxTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheIncrementTxTest.java
@@ -36,6 +36,7 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 
@@ -88,6 +89,9 @@ public class IgniteCacheIncrementTxTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testIncrementTxTopologyChange1() throws Exception {
+        if (MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-10264");
+
         nodeJoin(cacheConfiguration(1));
     }
 
@@ -95,6 +99,9 @@ public class IgniteCacheIncrementTxTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testIncrementTxTopologyChange2() throws Exception {
+        if (MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-10264");
+
         nodeJoin(cacheConfiguration(2));
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNoSyncForGetTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNoSyncForGetTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNoSyncForGetTest.java
index b14fecc..6324df4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNoSyncForGetTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNoSyncForGetTest.java
@@ -47,6 +47,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 
 /**
@@ -103,6 +104,13 @@ public class IgniteCacheNoSyncForGetTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxGet() throws Exception {
+        getTest(TRANSACTIONAL_SNAPSHOT);
+    }
+
+    /**
      * @param atomicityMode Cache atomicity mode.
      * @throws Exception If failed.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheStartFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheStartFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheStartFailoverTest.java
index a2d9da7..fa7d0a8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheStartFailoverTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheStartFailoverTest.java
@@ -56,6 +56,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 
 /**
@@ -82,6 +83,13 @@ public class IgniteClientCacheStartFailoverTest extends GridCommonAbstractTest {
     }
 
     /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        client = false;
+    }
+
+    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         stopAllGrids();
 
@@ -103,6 +111,13 @@ public class IgniteClientCacheStartFailoverTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testClientStartCoordinatorFailsMvccTx() throws Exception {
+        clientStartCoordinatorFails(TRANSACTIONAL_SNAPSHOT);
+    }
+
+    /**
      * @param atomicityMode Cache atomicity mode.
      * @throws Exception If failed.
      */
@@ -164,6 +179,16 @@ public class IgniteClientCacheStartFailoverTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testClientStartLastServerFailsMvccTx() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-10262");
+
+        clientStartLastServerFails(TRANSACTIONAL_SNAPSHOT);
+    }
+
+
+    /**
      * @param atomicityMode Cache atomicity mode.
      * @throws Exception If failed.
      */
@@ -547,6 +572,18 @@ public class IgniteClientCacheStartFailoverTest extends GridCommonAbstractTest {
             cache.putAll(map);
         }
 
+        //TODO: uncomment TRANSACTIONAL_SNAPSHOT cache creation when IGNITE-9470 will be fixed.
+       /* for (int i = 0; i < 3; i++) {
+            CacheConfiguration<Object, Object> ccfg = cacheConfiguration("mvcc-" + i, TRANSACTIONAL_SNAPSHOT, i);
+
+            IgniteCache<Object, Object> cache = node.createCache(ccfg);
+
+            cacheNames.add(ccfg.getName());
+
+            cache.putAll(map);
+        }*/
+
+
         return cacheNames;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMvccTxMultiThreadedAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMvccTxMultiThreadedAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMvccTxMultiThreadedAbstractTest.java
new file mode 100644
index 0000000..9e4609b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMvccTxMultiThreadedAbstractTest.java
@@ -0,0 +1,121 @@
+/*
+ * 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.internal.processors.cache;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Tests for local transactions.
+ */
+public abstract class IgniteMvccTxMultiThreadedAbstractTest extends IgniteTxAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9470");
+
+        super.beforeTestsStarted();
+    }
+
+    /**
+     * @return Thread count.
+     */
+    protected abstract int threadCount();
+
+    /**
+     /**
+     * @throws IgniteCheckedException If test failed.
+     */
+    public void testPessimisticRepeatableReadCommitMultithreaded() throws Exception {
+        checkCommitMultithreaded(PESSIMISTIC, REPEATABLE_READ);
+
+        finalChecks();
+    }
+
+    /**
+     * @throws IgniteCheckedException If test failed.
+     */
+    public void testPessimisticRepeatableReadRollbackMultithreaded() throws Exception {
+        checkRollbackMultithreaded(PESSIMISTIC, REPEATABLE_READ);
+
+        finalChecks();
+    }
+
+    /**
+     * @param concurrency Concurrency.
+     * @param isolation Isolation.
+     * @throws Exception If check failed.
+     */
+    protected void checkCommitMultithreaded(final TransactionConcurrency concurrency,
+        final TransactionIsolation isolation) throws Exception {
+        GridTestUtils.runMultiThreaded(new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                Thread t = Thread.currentThread();
+
+                t.setName(t.getName() + "-id-" + t.getId());
+
+                info("Starting commit thread: " + Thread.currentThread().getName());
+
+                try {
+                    checkCommit(concurrency, isolation);
+                }
+                finally {
+                    info("Finished commit thread: " + Thread.currentThread().getName());
+                }
+
+                return null;
+            }
+        }, threadCount(), concurrency + "-" + isolation);
+    }
+
+    /**
+     * @param concurrency Concurrency.
+     * @param isolation Isolation.
+     * @throws Exception If check failed.
+     */
+    protected void checkRollbackMultithreaded(final TransactionConcurrency concurrency,
+        final TransactionIsolation isolation) throws Exception {
+        final ConcurrentMap<Integer, String> map = new ConcurrentHashMap<>();
+        GridTestUtils.runMultiThreaded(new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                Thread t = Thread.currentThread();
+
+                t.setName(t.getName() + "-id-" + t.getId());
+
+                info("Starting rollback thread: " + Thread.currentThread().getName());
+
+                try {
+                    checkRollback(map, concurrency, isolation);
+
+                    return null;
+                }
+                finally {
+                    info("Finished rollback thread: " + Thread.currentThread().getName());
+                }
+            }
+        }, threadCount(), concurrency + "-" + isolation);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMvccTxSingleThreadedAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMvccTxSingleThreadedAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMvccTxSingleThreadedAbstractTest.java
new file mode 100644
index 0000000..b9e7cee
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMvccTxSingleThreadedAbstractTest.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.IgniteCheckedException;
+
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Tests for local transactions.
+ */
+public abstract class IgniteMvccTxSingleThreadedAbstractTest extends IgniteTxAbstractTest {
+    /**
+     * @throws IgniteCheckedException If test failed.
+     */
+    public void testPessimisticRepeatableReadCommit() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-10261");
+
+        checkCommit(PESSIMISTIC, REPEATABLE_READ);
+
+        finalChecks();
+    }
+
+    /**
+     * @throws IgniteCheckedException If test failed.
+     */
+    public void testPessimisticRepeatableReadRollback() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-10261");
+
+        checkRollback(PESSIMISTIC, REPEATABLE_READ);
+
+        finalChecks();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteNearClientCacheCloseTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteNearClientCacheCloseTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteNearClientCacheCloseTest.java
index e7ab805..0ca0f3e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteNearClientCacheCloseTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteNearClientCacheCloseTest.java
@@ -37,10 +37,12 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 
 /**
@@ -108,6 +110,26 @@ public class IgniteNearClientCacheCloseTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testNearCacheCloseMvccTx1() throws Exception {
+        nearCacheClose(1, false, TRANSACTIONAL_SNAPSHOT);
+
+        if (MvccFeatureChecker.isSupported(MvccFeatureChecker.Feature.NEAR_CACHE))
+            nearCacheClose(1, true, TRANSACTIONAL_SNAPSHOT);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNearCacheCloseMvccTx2() throws Exception {
+        nearCacheClose(4, false, TRANSACTIONAL_SNAPSHOT);
+
+        if (MvccFeatureChecker.isSupported(MvccFeatureChecker.Feature.NEAR_CACHE))
+            nearCacheClose(4, true, TRANSACTIONAL_SNAPSHOT);
+    }
+
+    /**
      * @param srvs Number of server nodes.
      * @param srvNearCache {@code True} to enable near cache on server nodes.
      * @param atomicityMode Cache atomicity mode.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java
index 1830db0..3ae4224 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java
@@ -57,7 +57,7 @@ abstract class IgniteTxAbstractTest extends GridCommonAbstractTest {
     private static final AtomicInteger cntr = new AtomicInteger();
 
     /** */
-    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+    private final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /**
      * Start grid by default.
@@ -115,12 +115,16 @@ abstract class IgniteTxAbstractTest extends GridCommonAbstractTest {
             info(msg);
     }
 
-    /**
-     * @throws Exception If failed.
-     */
+    /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
-        for (int i = 0; i < gridCount(); i++)
-            startGrid(i);
+        startGridsMultiThreaded(gridCount());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiThreadedAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiThreadedAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiThreadedAbstractTest.java
index 9a385a3..65dc2e7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiThreadedAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiThreadedAbstractTest.java
@@ -85,7 +85,6 @@ public abstract class IgniteTxMultiThreadedAbstractTest extends IgniteTxAbstract
     protected void checkRollbackMultithreaded(final TransactionConcurrency concurrency,
         final TransactionIsolation isolation) throws Exception {
         final ConcurrentMap<Integer, String> map = new ConcurrentHashMap<>();
-
         GridTestUtils.runMultiThreaded(new Callable<Object>() {
             @Nullable @Override public Object call() throws Exception {
                 Thread t = Thread.currentThread();

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
index 5cb206f..10d04bd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
@@ -49,6 +49,7 @@ import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
@@ -80,6 +81,13 @@ public class CacheLoadingConcurrentGridStartSelfTest extends GridCommonAbstractT
     protected volatile boolean restarts;
 
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_STORE);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
index 852e855..844a430 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
@@ -36,6 +36,7 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 
@@ -56,6 +57,13 @@ public class CacheLockReleaseNodeLeaveTest extends GridCommonAbstractTest {
     private static final String REPLICATED_TEST_CACHE = "REPLICATED_TEST_CACHE";
 
     /** {@inheritDoc} */
+    @Override public void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractPrimarySyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractPrimarySyncSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractPrimarySyncSelfTest.java
index 625cb18..f6b9302 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractPrimarySyncSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractPrimarySyncSelfTest.java
@@ -24,6 +24,7 @@ import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 
@@ -70,6 +71,9 @@ public abstract class GridCacheAbstractPrimarySyncSelfTest extends GridCommonAbs
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
+        if (nearConfiguration() != null)
+            MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
         assert GRID_CNT > 1;
 
         startGrids(GRID_CNT);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheBasicOpAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheBasicOpAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheBasicOpAbstractTest.java
index a55ff2d..0398813 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheBasicOpAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheBasicOpAbstractTest.java
@@ -32,6 +32,7 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 
@@ -73,6 +74,11 @@ public abstract class GridCacheBasicOpAbstractTest extends GridCommonAbstractTes
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_EVENTS);
+
+        if (MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-7952");
+
         startGridsMultiThreaded(3);
 
         ignite1 = grid(0);
@@ -327,6 +333,8 @@ public abstract class GridCacheBasicOpAbstractTest extends GridCommonAbstractTes
      * @throws Exception In case of error.
      */
     public void testPutWithExpiration() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.EXPIRATION);
+
         IgniteCache<String, String> cache1 = ignite1.cache(DEFAULT_CACHE_NAME);
         IgniteCache<String, String> cache2 = ignite2.cache(DEFAULT_CACHE_NAME);
         IgniteCache<String, String> cache3 = ignite3.cache(DEFAULT_CACHE_NAME);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLockAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLockAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLockAbstractTest.java
index 48b9a20..5909213 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLockAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLockAbstractTest.java
@@ -38,6 +38,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestThread;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
@@ -65,6 +66,13 @@ public abstract class GridCacheLockAbstractTest extends GridCommonAbstractTest {
     /** Ip-finder. */
     private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
+        super.setUp();
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeAbstractTest.java
index 1d56ab6..912aece 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeAbstractTest.java
@@ -35,6 +35,7 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
@@ -87,6 +88,8 @@ public abstract class GridCacheMultiNodeAbstractTest extends GridCommonAbstractT
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_EVENTS);
+
         ignite1 = startGrid(1);
         ignite2 = startGrid(2);
         ignite3 = startGrid(3);
@@ -228,6 +231,8 @@ public abstract class GridCacheMultiNodeAbstractTest extends GridCommonAbstractT
      * @throws Exception If test failed.
      */
     public void testLockUnlock() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
         CacheEventListener lockLsnr1 = new CacheEventListener(ignite1, new CountDownLatch(1), EVT_CACHE_OBJECT_LOCKED);
 
         addListener(ignite1, lockLsnr1, EVT_CACHE_OBJECT_LOCKED);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeLockAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeLockAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeLockAbstractTest.java
index cc3b894..6636196 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeLockAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeLockAbstractTest.java
@@ -38,6 +38,7 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestThread;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
@@ -63,6 +64,14 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
     /** Listeners. */
     private static Collection<IgnitePredicate<Event>> lsnrs = new ArrayList<>();
 
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_EVENTS);
+
+        super.setUp();
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java
index 8de2d79..6375e3e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java
@@ -37,6 +37,7 @@ import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -75,6 +76,13 @@ public abstract class GridCacheNodeFailureAbstractTest extends GridCommonAbstrac
     /** Grid instances. */
     private static final List<Ignite> IGNITEs = new ArrayList<>();
 
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
+        super.setUp();
+    }
+
     /**
      * Start grid by default.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
index 334608d..f5695ca 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
@@ -40,6 +40,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.GridTestUtils.SF;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.util.TestTcpCommunicationSpi;
 
@@ -57,6 +58,13 @@ public class GridCachePartitionNotLoadedEventSelfTest extends GridCommonAbstract
     private int backupCnt;
 
     /** {@inheritDoc} */
+    @Override public void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_EVENTS);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
@@ -101,10 +109,7 @@ public class GridCachePartitionNotLoadedEventSelfTest extends GridCommonAbstract
     public void testPrimaryAndBackupDead() throws Exception {
         backupCnt = 1;
 
-        startGrid(0);
-        startGrid(1);
-        startGrid(2);
-        startGrid(3);
+        startGridsMultiThreaded(4);
 
         awaitPartitionMapExchange();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedNearDisabledMvccTxMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedNearDisabledMvccTxMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedNearDisabledMvccTxMultiThreadedSelfTest.java
new file mode 100644
index 0000000..cf8115c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedNearDisabledMvccTxMultiThreadedSelfTest.java
@@ -0,0 +1,31 @@
+/*
+ * 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.internal.processors.cache.distributed;
+
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMvccTxMultiThreadedSelfTest;
+
+/**
+ *
+ */
+public class GridCachePartitionedNearDisabledMvccTxMultiThreadedSelfTest
+    extends GridCachePartitionedMvccTxMultiThreadedSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean nearEnabled() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadRestartAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadRestartAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadRestartAbstractSelfTest.java
index 2ea1ed8..8ef5fb8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadRestartAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadRestartAbstractSelfTest.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -97,6 +98,14 @@ public abstract class GridCachePreloadRestartAbstractSelfTest extends GridCommon
     private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        if (nearEnabled())
+            MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
 
@@ -197,6 +206,9 @@ public abstract class GridCachePreloadRestartAbstractSelfTest extends GridCommon
      * @throws Exception If failed.
      */
     public void testDisabledPreloadRestart() throws Exception {
+        if (MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-10261");
+
         preloadMode = NONE;
 
         checkRestart();

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java
index 913321e..8cb0885 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java
@@ -49,6 +49,7 @@ import org.apache.ignite.transactions.TransactionIsolation;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
@@ -74,6 +75,9 @@ public class GridCacheTransformEventSelfTest extends GridCommonAbstractTest {
     /** Cache name. */
     private static final String CACHE_NAME = "cache";
 
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
     /** Key 1. */
     private Integer key1;
 
@@ -83,9 +87,6 @@ public class GridCacheTransformEventSelfTest extends GridCommonAbstractTest {
     /** Two keys in form of a set. */
     private Set<Integer> keys;
 
-    /** IP finder. */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
     /** Nodes. */
     private Ignite[] ignites;
 
@@ -328,6 +329,17 @@ public class GridCacheTransformEventSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Test TRANSACTIONAL_SNAPSHOT LOCAL cache with PESSIMISTIC/REPEATABLE_READ transaction.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMvccTxLocalPessimisticRepeatableRead() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9530");
+
+        checkMvccTx(LOCAL, PESSIMISTIC, REPEATABLE_READ);
+    }
+
+    /**
      * Test TRANSACTIONAL PARTITIONED cache with OPTIMISTIC/REPEATABLE_READ transaction.
      *
      * @throws Exception If failed.
@@ -382,6 +394,18 @@ public class GridCacheTransformEventSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Test TRANSACTIONAL_SNAPSHOT PARTITIONED cache with PESSIMISTIC/REPEATABLE_READ transaction.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMvccTxPartitionedPessimisticRepeatableRead() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9321");
+
+        checkMvccTx(PARTITIONED, PESSIMISTIC, REPEATABLE_READ);
+    }
+
+
+    /**
      * Test TRANSACTIONAL REPLICATED cache with OPTIMISTIC/REPEATABLE_READ transaction.
      *
      * @throws Exception If failed.
@@ -436,6 +460,17 @@ public class GridCacheTransformEventSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Test TRANSACTIONAL_SNAPSHOT REPLICATED cache with PESSIMISTIC/REPEATABLE_READ transaction.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMvccTxReplicatedPessimisticRepeatableRead() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9321");
+
+        checkMvccTx(REPLICATED, PESSIMISTIC, REPEATABLE_READ);
+    }
+
+    /**
      * Test ATOMIC LOCAL cache.
      *
      * @throws Exception If failed.
@@ -495,6 +530,21 @@ public class GridCacheTransformEventSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Check TRANSACTIONAL_SNAPSHOT cache.
+     *
+     * @param cacheMode Cache mode.
+     * @param txConcurrency TX concurrency.
+     * @param txIsolation TX isolation.
+     * @throws Exception If failed.
+     */
+    private void checkMvccTx(CacheMode cacheMode, TransactionConcurrency txConcurrency,
+        TransactionIsolation txIsolation) throws Exception {
+        initialize(cacheMode, TRANSACTIONAL_SNAPSHOT, txConcurrency, txIsolation);
+
+        checkTx0();
+    }
+
+    /**
      * Check TRANSACTIONAL cache.
      *
      * @param cacheMode Cache mode.
@@ -506,6 +556,14 @@ public class GridCacheTransformEventSelfTest extends GridCommonAbstractTest {
         TransactionIsolation txIsolation) throws Exception {
         initialize(cacheMode, TRANSACTIONAL, txConcurrency, txIsolation);
 
+        checkTx0();
+    }
+
+    /**
+     * Check TX cache.
+     */
+    private void checkTx0()  {
+
         System.out.println("BEFORE: " + evts.size());
 
         caches[0].invoke(key1, new Transformer());


[38/50] [abbrv] ignite git commit: IGNITE-10106 Optimize several Cache 5 tests by scale factor - Fixes #5252.

Posted by ag...@apache.org.
IGNITE-10106 Optimize several Cache 5 tests by scale factor - Fixes #5252.

Signed-off-by: Pavel Kovalenko <jo...@gmail.com>


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

Branch: refs/heads/ignite-9720
Commit: 9d3c757355980fae9b558b0c2e736265adc028a0
Parents: 0b4282b
Author: Alexey Platonov <ap...@gmail.com>
Authored: Tue Nov 27 19:55:06 2018 +0300
Committer: Pavel Kovalenko <jo...@gmail.com>
Committed: Tue Nov 27 19:55:06 2018 +0300

----------------------------------------------------------------------
 .../CacheSerializableTransactionsTest.java      | 32 +++++++++-----------
 .../IgniteCacheStarvationOnRebalanceTest.java   |  5 +--
 2 files changed, 17 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9d3c7573/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java
index 714ae6a..0afd4ba 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java
@@ -71,6 +71,7 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.GridTestUtils.SF;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -3529,9 +3530,9 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
 
                     IgniteTransactions txs = ignite.transactions();
 
-                    final int KEYS = 100;
+                    final int KEYS = SF.apply(100);
 
-                    for (int i = 0; i < 1000; i++) {
+                    for (int i = 0; i < SF.apply(1000); i++) {
                         Integer key1 = rnd.nextInt(KEYS);
 
                         Integer key2;
@@ -3671,14 +3672,14 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
 
             final long stopTime = U.currentTimeMillis() + getTestTimeout() - 30_000;
 
-            for (int i = 0; i < 30; i++) {
+            for (int i = 0; i < SF.apply(30); i++) {
                 final AtomicInteger cntr = new AtomicInteger();
 
                 final Integer key = i;
 
                 final AtomicInteger threadIdx = new AtomicInteger();
 
-                final int THREADS = 10;
+                final int THREADS = SF.applyLB(10, 2);
 
                 final CyclicBarrier barrier = new CyclicBarrier(THREADS);
 
@@ -3696,7 +3697,7 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
 
                         barrier.await();
 
-                        for (int i = 0; i < 1000; i++) {
+                        for (int i = 0; i < SF.apply(1000); i++) {
                             if (i % 100 == 0 && U.currentTimeMillis() > stopTime)
                                 break;
 
@@ -3911,7 +3912,7 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
                     caches.add(client.<Integer, Integer>cache(cacheName));
             }
 
-            for (int i = 0; i < 100; i++) {
+            for (int i = 0; i < SF.apply(100); i++) {
                 if (U.currentTimeMillis() > stopTime)
                     break;
 
@@ -3921,7 +3922,7 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
 
                 final AtomicInteger threadIdx = new AtomicInteger();
 
-                final int THREADS = 10;
+                final int THREADS = SF.applyLB(10, 2);
 
                 final CyclicBarrier barrier = new CyclicBarrier(THREADS);
 
@@ -3943,7 +3944,7 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
 
                         barrier.await();
 
-                        for (int i = 0; i < 50; i++) {
+                        for (int i = 0; i < SF.apply(50); i++) {
                             while (true) {
                                 try {
                                     ThreadLocalRandom rnd = ThreadLocalRandom.current();
@@ -3963,10 +3964,8 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
                                         tx.commit();
 
                                         if (rmv) {
-                                            if (val != null) {
-                                                for (int j = 0; j < val; j++)
-                                                    cntr.decrementAndGet();
-                                            }
+                                            if (val != null)
+                                                cntr.getAndUpdate(x -> x - val);
                                         }
                                         else
                                             cntr.incrementAndGet();
@@ -4669,9 +4668,6 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
         final boolean restart,
         final boolean nonSer
     ) throws Exception {
-        if (FAST)
-            return;
-
         assert !updateNodes.isEmpty();
 
         final Ignite srv = ignite(1);
@@ -4680,17 +4676,17 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
             srv.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 1, false, false)).getName();
 
         try {
-            final int KEYS = 100;
+            final int KEYS = SF.apply(20);
 
             final AtomicBoolean finished = new AtomicBoolean();
 
             IgniteInternalFuture<?> fut = restart ? restartFuture(finished, null) : null;
 
             try {
-                for (int i = 0; i < 10; i++) {
+                for (int i = 0; i < SF.applyLB(10, 2); i++) {
                     log.info("Iteration: " + i);
 
-                    final long stopTime = U.currentTimeMillis() + 10_000;
+                    final long stopTime = U.currentTimeMillis() + SF.applyLB(10_000, 1_000);
 
                     final AtomicInteger idx = new AtomicInteger();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d3c7573/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheStarvationOnRebalanceTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheStarvationOnRebalanceTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheStarvationOnRebalanceTest.java
index 621d10d..85d463f 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheStarvationOnRebalanceTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheStarvationOnRebalanceTest.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.GridTestUtils.SF;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
@@ -89,7 +90,7 @@ public class IgniteCacheStarvationOnRebalanceTest extends GridCacheAbstractSelfT
     public void testLoadSystemWithPutAndStartRebalancing() throws Exception {
         final IgniteCache<Integer, CacheValue> cache = grid(0).cache(DEFAULT_CACHE_NAME);
 
-        final long endTime = System.currentTimeMillis() + TEST_TIMEOUT - 60_000;
+        final long endTime = System.currentTimeMillis() + SF.applyLB((int)TEST_TIMEOUT - 60_000, 5_000);
 
         int iter = 0;
 
@@ -99,7 +100,7 @@ public class IgniteCacheStarvationOnRebalanceTest extends GridCacheAbstractSelfT
             final AtomicBoolean stop = new AtomicBoolean();
 
             IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
-                @Override public Void call() throws Exception {
+                @Override public Void call() {
                     ThreadLocalRandom rnd = ThreadLocalRandom.current();
 
                     while (!stop.get() && System.currentTimeMillis() < endTime) {


[47/50] [abbrv] ignite git commit: IGNITE-10432: Fixed data types in world.sql.

Posted by ag...@apache.org.
IGNITE-10432: Fixed data types in world.sql.


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

Branch: refs/heads/ignite-9720
Commit: d21f86f9e7817481acbb9438a9e07ca605246807
Parents: 3606b21
Author: devozerov <vo...@gridgain.com>
Authored: Wed Nov 28 11:11:20 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Nov 28 11:11:20 2018 +0300

----------------------------------------------------------------------
 examples/sql/world.sql | 28 ++++++++++++++--------------
 1 file changed, 14 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d21f86f9/examples/sql/world.sql
----------------------------------------------------------------------
diff --git a/examples/sql/world.sql b/examples/sql/world.sql
index a34ee71..829fba1 100644
--- a/examples/sql/world.sql
+++ b/examples/sql/world.sql
@@ -2,30 +2,30 @@ DROP TABLE IF EXISTS Country;
 
 CREATE TABLE Country (
   Code CHAR(3) PRIMARY KEY,
-  Name CHAR(52),
-  Continent CHAR(50),
-  Region CHAR(26),
+  Name VARCHAR,
+  Continent VARCHAR,
+  Region VARCHAR,
   SurfaceArea DECIMAL(10,2),
-  IndepYear SMALLINT(6),
-  Population INT(11),
+  IndepYear SMALLINT,
+  Population INT,
   LifeExpectancy DECIMAL(3,1),
   GNP DECIMAL(10,2),
   GNPOld DECIMAL(10,2),
-  LocalName CHAR(45),
-  GovernmentForm CHAR(45),
-  HeadOfState CHAR(60),
-  Capital INT(11),
+  LocalName VARCHAR,
+  GovernmentForm VARCHAR,
+  HeadOfState VARCHAR,
+  Capital INT,
   Code2 CHAR(2)
 ) WITH "template=partitioned, backups=1, CACHE_NAME=Country, VALUE_TYPE=demo.model.Country";
 
 DROP TABLE IF EXISTS City;
 
 CREATE TABLE City (
-  ID INT(11),
-  Name CHAR(35),
+  ID INT,
+  Name VARCHAR,
   CountryCode CHAR(3),
-  District CHAR(20),
-  Population INT(11),
+  District VARCHAR,
+  Population INT,
   PRIMARY KEY (ID, CountryCode)
 ) WITH "template=partitioned, backups=1, affinityKey=CountryCode, CACHE_NAME=City, KEY_TYPE=demo.model.CityKey, VALUE_TYPE=demo.model.City";
 
@@ -35,7 +35,7 @@ DROP TABLE IF EXISTS CountryLanguage;
 
 CREATE TABLE CountryLanguage (
   CountryCode CHAR(3),
-  Language CHAR(30),
+  Language VARCHAR,
   IsOfficial CHAR(2),
   Percentage DECIMAL(4,1),
   PRIMARY KEY (CountryCode, Language)


[44/50] [abbrv] ignite git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-9720

Posted by ag...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-9720


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

Branch: refs/heads/ignite-9720
Commit: 0063fb1d7addad2407598ee56bcb92a473dde9dc
Parents: ceacab0 c3fd4a9
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Wed Nov 28 10:31:37 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Nov 28 10:31:37 2018 +0300

----------------------------------------------------------------------
 ...niteFunctionDistributedInferenceExample.java |  100 +
 .../TensorFlowDistributedInferenceExample.java  |   99 +
 .../TensorFlowLocalInferenceExample.java        |   85 +
 .../TensorFlowThreadedInferenceExample.java     |   95 +
 .../examples/ml/inference/package-info.java     |   22 +
 .../ml/preprocessing/StandardScalerExample.java |   84 +
 .../examples/ml/util/MLSandboxDatasets.java     |   16 +-
 .../ignite/examples/ml/util/SandboxMLCache.java |   11 +-
 .../ignite/examples/ml/util/datasets/README.md  |    2 -
 .../ml/util/datasets/boston_housing_dataset.txt |  505 ----
 .../ml/util/datasets/cleared_machines.csv       |  209 --
 .../ml/util/datasets/glass_identification.csv   |  116 -
 .../ignite/examples/ml/util/datasets/iris.txt   |  150 --
 .../examples/ml/util/datasets/mortalitydata.csv |   53 -
 .../examples/ml/util/datasets/titanic.csv       | 1310 ---------
 .../ml/util/datasets/titanic_10_rows.csv        |   11 -
 .../ml/util/datasets/two_classed_iris.csv       |  100 -
 .../ignite/examples/ml/util/datasets/wine.txt   |  178 --
 examples/src/main/resources/datasets/README.md  |    2 +
 .../datasets/boston_housing_dataset.txt         |  505 ++++
 .../resources/datasets/cleared_machines.csv     |  209 ++
 .../resources/datasets/glass_identification.csv |  116 +
 examples/src/main/resources/datasets/iris.txt   |  150 ++
 .../main/resources/datasets/mortalitydata.csv   |   53 +
 .../resources/datasets/t10k-images-idx3-ubyte   |  Bin 0 -> 9876027 bytes
 .../resources/datasets/t10k-labels-idx1-ubyte   |  Bin 0 -> 10008 bytes
 .../src/main/resources/datasets/titanic.csv     | 1310 +++++++++
 .../main/resources/datasets/titanic_10_rows.csv |   11 +
 .../resources/datasets/two_classed_iris.csv     |  100 +
 examples/src/main/resources/datasets/wine.txt   |  178 ++
 .../resources/ml/mnist_tf_model/saved_model.pb  |  Bin 0 -> 37185 bytes
 .../variables/variables.data-00000-of-00001     |  Bin 0 -> 13098544 bytes
 .../ml/mnist_tf_model/variables/variables.index |  Bin 0 -> 410 bytes
 .../ignite/examples/BasicExamplesSelfTest.java  |    6 +
 .../examples/CacheClientBinaryExampleTest.java  |    3 +
 .../CacheContinuousQueryExamplesSelfTest.java   |    4 +
 .../ignite/examples/CacheExamplesSelfTest.java  |    4 +
 .../ComputeClientBinaryExampleTest.java         |    2 +
 .../examples/EncryptedCacheExampleSelfTest.java |    3 +
 .../ignite/examples/EventsExamplesSelfTest.java |    2 +
 .../examples/MessagingExamplesSelfTest.java     |    3 +
 .../examples/SpringDataExampleSelfTest.java     |    2 +
 .../ignite/examples/SqlExamplesSelfTest.java    |    4 +
 .../IgniteExamplesJ8SelfTestSuite.java          |   53 +-
 .../testsuites/IgniteExamplesSelfTestSuite.java |   59 +-
 idea/ignite_inspections.xml                     |    2 +-
 .../jdbc2/JdbcDistributedJoinsQueryTest.java    |    4 -
 .../JettyRestProcessorAbstractSelfTest.java     |   90 +-
 ...RestProcessorAuthenticationAbstractTest.java |    6 +-
 ...tingToWalV2SerializerWithCompactionTest.java |    5 -
 .../PersistenceBasicCompatibilityTest.java      |    5 -
 modules/compress/licenses/Snappy-LICENSE.txt    |   29 +
 .../compress/licenses/Snappy-java-LICENSE.txt   |  202 ++
 modules/compress/licenses/Zstd-LICENSE.txt      |   30 +
 modules/compress/licenses/Zstd-jni-LICENSE.txt  |   26 +
 modules/compress/licenses/jnr-ffi-LICENSE.txt   |   25 +
 modules/compress/licenses/jnr-posix-LICENSE.txt | 2486 ++++++++++++++++++
 modules/compress/licenses/lz4-LICENSE.txt       |   11 +
 modules/compress/licenses/lz4-java-LICENSE.txt  |  202 ++
 modules/compress/pom.xml                        |  113 +
 .../compress/CompressionProcessorImpl.java      |  403 +++
 .../compress/NativeFileSystemLinux.java         |  142 +
 .../compress/NativeFileSystemPosix.java         |   54 +
 .../compress/CompressionProcessorTest.java      | 1021 +++++++
 ...DiskPageCompressionIntegrationAsyncTest.java |   30 +
 .../DiskPageCompressionIntegrationTest.java     |  468 ++++
 .../compress/FileSystemUtilsTest.java           |  193 ++
 .../IgnitePdsCompressionTestSuite.java          |   56 +
 .../IgnitePdsCompressionTestSuite2.java         |   38 +
 .../org/apache/ignite/DataStorageMetrics.java   |   18 +
 .../apache/ignite/IgniteSystemProperties.java   |   62 +-
 .../cache/affinity/AffinityKeyMapped.java       |    4 +-
 .../configuration/CacheConfiguration.java       |   60 +
 .../configuration/DataStorageConfiguration.java |   16 +-
 .../configuration/DiskPageCompression.java      |   38 +
 .../ignite/internal/GridKernalContext.java      |    6 +
 .../ignite/internal/GridKernalContextImpl.java  |   12 +
 .../ignite/internal/IgniteComponentType.java    |    7 +
 .../apache/ignite/internal/IgniteKernal.java    |    2 +
 .../ignite/internal/IgniteNodeAttributes.java   |    3 -
 .../internal/commandline/CommandHandler.java    |   50 +-
 .../internal/managers/discovery/DiscoCache.java |   15 -
 .../discovery/GridDiscoveryManager.java         |   38 +-
 .../internal/pagemem/store/PageStore.java       |   34 +-
 .../processors/affinity/AffinityAssignment.java |    6 -
 .../affinity/GridAffinityAssignment.java        |   15 +-
 .../affinity/GridAffinityAssignmentCache.java   |   22 +-
 .../affinity/GridAffinityProcessor.java         |    2 +-
 .../processors/affinity/GridAffinityUtils.java  |    2 +-
 .../affinity/HistoryAffinityAssignment.java     |   19 +-
 .../cache/CacheAffinitySharedManager.java       |   13 +-
 .../cache/CacheCompressionManager.java          |   95 +
 .../cache/CacheEntryPredicateContainsValue.java |    4 +
 .../cache/CacheGroupMetricsMXBeanImpl.java      |   23 +-
 .../processors/cache/ExchangeContext.java       |   43 +-
 .../processors/cache/GridCacheAdapter.java      |   20 +-
 .../cache/GridCacheAffinityManager.java         |    5 -
 .../processors/cache/GridCacheContext.java      |   13 +
 .../processors/cache/GridCacheMapEntry.java     |    8 +-
 .../GridCachePartitionExchangeManager.java      |   12 +-
 .../processors/cache/GridCacheProcessor.java    |    3 +
 .../cache/IgniteCacheOffheapManager.java        |    5 +
 .../cache/IgniteCacheOffheapManagerImpl.java    |   18 +
 .../processors/cache/WalStateManager.java       |   24 +
 .../CacheDataStructuresManager.java             |   15 +-
 .../distributed/dht/GridDhtCacheEntry.java      |    8 +-
 .../dht/GridDhtTxAbstractEnlistFuture.java      |  169 +-
 .../distributed/dht/GridDhtTxEnlistFuture.java  |   13 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   23 -
 .../dht/atomic/GridDhtAtomicCache.java          |   30 +-
 .../GridNearAtomicAbstractUpdateFuture.java     |   43 +-
 .../dht/colocated/GridDhtColocatedCache.java    |    6 +-
 .../GridDhtPartitionsExchangeFuture.java        |   79 +-
 .../GridDhtPartitionsSingleMessage.java         |   68 +-
 .../topology/GridClientPartitionTopology.java   |    7 -
 .../dht/topology/GridDhtLocalPartition.java     |   33 +-
 .../dht/topology/GridDhtPartitionTopology.java  |    4 -
 .../topology/GridDhtPartitionTopologyImpl.java  |   17 +-
 ...arOptimisticSerializableTxPrepareFuture.java |   35 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   39 +-
 ...ridNearOptimisticTxPrepareFutureAdapter.java |  114 -
 .../GridNearPessimisticTxPrepareFuture.java     |   78 +-
 .../near/GridNearTxEnlistFuture.java            |   22 +-
 .../cache/distributed/near/GridNearTxLocal.java |   30 +-
 .../near/GridNearTxPrepareRequest.java          |   17 -
 .../near/GridNearTxPrepareResponse.java         |   52 +-
 .../GridNearTxQueryResultsEnlistFuture.java     |   21 +-
 .../mvcc/MvccPreviousCoordinatorQueries.java    |   13 +-
 .../processors/cache/mvcc/MvccProcessor.java    |   45 +-
 .../cache/mvcc/MvccProcessorImpl.java           |  383 +--
 .../cache/mvcc/MvccQueryTrackerImpl.java        |   12 +-
 .../processors/cache/mvcc/MvccUtils.java        |   48 +-
 .../persistence/DataStorageMetricsImpl.java     |   22 +-
 .../persistence/DataStorageMetricsSnapshot.java |   18 +
 .../GridCacheDatabaseSharedManager.java         |   91 +-
 .../persistence/GridCacheOffheapManager.java    |   14 +-
 .../cache/persistence/file/AbstractFileIO.java  |    2 +-
 .../cache/persistence/file/AsyncFileIO.java     |   38 +-
 .../cache/persistence/file/EncryptedFileIO.java |   16 +-
 .../cache/persistence/file/FileIO.java          |   19 +
 .../cache/persistence/file/FileIODecorator.java |   17 +-
 .../cache/persistence/file/FilePageStore.java   |   72 +-
 .../persistence/file/FilePageStoreManager.java  |   65 +-
 .../cache/persistence/file/FilePageStoreV2.java |   21 +
 .../persistence/file/RandomAccessFileIO.java    |   35 +
 .../cache/persistence/file/UnzipFileIO.java     |   15 +
 .../persistence/pagemem/PageMemoryImpl.java     |   29 +-
 .../cache/persistence/tree/BPlusTree.java       |   36 +
 .../persistence/tree/io/AbstractDataPageIO.java |   79 +-
 .../cache/persistence/tree/io/BPlusIO.java      |   32 +-
 .../persistence/tree/io/CompactablePageIO.java  |   43 +
 .../persistence/tree/io/DataPagePayload.java    |   16 +
 .../cache/persistence/tree/io/PageIO.java       |  103 +-
 .../wal/reader/StandaloneGridKernalContext.java |    6 +
 .../wal/serializer/RecordDataV1Serializer.java  |   17 +-
 .../cache/transactions/IgniteTxAdapter.java     |   46 +-
 .../cache/transactions/IgniteTxHandler.java     |  239 +-
 .../cache/transactions/IgniteTxManager.java     |    4 +-
 .../cache/tree/mvcc/data/MvccUpdateDataRow.java |   11 +-
 .../cluster/GridClusterStateProcessor.java      |    3 +-
 .../compress/CompressionProcessor.java          |  173 ++
 .../processors/compress/FileSystemUtils.java    |  129 +
 .../processors/compress/NativeFileSystem.java   |   50 +
 .../datastructures/GridCacheSetImpl.java        |    4 +-
 .../processors/query/GridQueryProcessor.java    |    1 -
 .../service/GridServiceProcessor.java           |   12 -
 .../apache/ignite/internal/util/GridUnsafe.java |    1 +
 .../ignite/internal/util/IgniteUtils.java       |  213 +-
 .../verify/VisorValidateIndexesJobResult.java   |    9 +
 .../verify/VisorValidateIndexesTaskArg.java     |   23 +-
 .../ignite/mxbean/CacheGroupMetricsMXBean.java  |   12 +
 .../ignite/mxbean/DataStorageMetricsMXBean.java |    8 +
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   41 -
 ...ndezvousAffinityFunctionSimpleBenchmark.java |   26 +-
 .../cache/store/CacheStoreWriteErrorTest.java   |  127 +
 .../GridMultithreadedJobStealingSelfTest.java   |    6 +-
 .../BinaryArrayIdentityResolverSelfTest.java    |    4 +-
 .../binary/BinaryFieldsOffheapSelfTest.java     |    4 +-
 .../BinaryFooterOffsetsOffheapSelfTest.java     |    4 +-
 .../BinarySerialiedFieldComparatorSelfTest.java |    4 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |   37 -
 .../cache/CacheConcurrentReadThroughTest.java   |    8 +
 .../cache/CacheEnumOperationsAbstractTest.java  |   18 +-
 .../CacheSerializableTransactionsTest.java      |   32 +-
 .../cache/CrossCacheTxRandomOperationsTest.java |   23 +-
 .../cache/GridCacheAbstractMetricsSelfTest.java |    6 -
 .../cache/GridCacheBasicApiAbstractTest.java    |   27 +
 .../cache/GridCacheBasicStoreAbstractTest.java  |    8 +
 .../GridCacheEvictionEventAbstractTest.java     |    9 +
 .../GridCacheFinishPartitionsSelfTest.java      |    8 +
 ...idCacheGetAndTransformStoreAbstractTest.java |    8 +
 .../cache/GridCacheOffheapUpdateSelfTest.java   |   17 +-
 .../cache/GridCachePartitionedGetSelfTest.java  |    6 +
 .../GridCacheVariableTopologySelfTest.java      |    9 +
 .../IgniteCacheEntryProcessorNodeJoinTest.java  |  111 +-
 .../cache/IgniteCacheIncrementTxTest.java       |    7 +
 .../cache/IgniteCacheNoSyncForGetTest.java      |    8 +
 .../IgniteClientCacheStartFailoverTest.java     |   37 +
 .../IgniteClusterActivateDeactivateTest.java    |    2 +-
 ...erActivateDeactivateTestWithPersistence.java |   37 +
 ...ynamicCacheStartFailWithPersistenceTest.java |    1 +
 .../IgniteMvccTxMultiThreadedAbstractTest.java  |  121 +
 .../IgniteMvccTxSingleThreadedAbstractTest.java |   50 +
 .../cache/IgniteNearClientCacheCloseTest.java   |   22 +
 .../processors/cache/IgniteTxAbstractTest.java  |   16 +-
 .../IgniteTxMultiThreadedAbstractTest.java      |    1 -
 .../cache/WalModeChangeAbstractSelfTest.java    |   45 +-
 .../GridCacheBinaryObjectsAbstractSelfTest.java |   58 +
 ...ntNodeBinaryObjectMetadataMultinodeTest.java |    4 +-
 .../GridCacheSetAbstractSelfTest.java           |   31 +-
 ...idCachePartitionedSetWithClientSelfTest.java |   52 +
 ...chePartitionedSetWithNodeFilterSelfTest.java |   37 +
 ...IgnitePartitionedCountDownLatchSelfTest.java |    5 -
 ...ridCacheReplicatedSetWithClientSelfTest.java |   52 +
 ...acheReplicatedSetWithNodeFilterSelfTest.java |   37 +
 .../distributed/CacheExchangeMergeTest.java     |    4 +-
 ...CacheLoadingConcurrentGridStartSelfTest.java |    8 +
 .../CacheLockReleaseNodeLeaveTest.java          |    8 +
 .../GridCacheAbstractPrimarySyncSelfTest.java   |    4 +
 .../GridCacheBasicOpAbstractTest.java           |    8 +
 .../distributed/GridCacheLockAbstractTest.java  |    8 +
 .../GridCacheMultiNodeAbstractTest.java         |    5 +
 .../GridCacheMultiNodeLockAbstractTest.java     |    9 +
 .../GridCacheNodeFailureAbstractTest.java       |    8 +
 ...ridCachePartitionNotLoadedEventSelfTest.java |   17 +-
 ...NearDisabledMvccTxMultiThreadedSelfTest.java |   31 +
 ...GridCachePreloadRestartAbstractSelfTest.java |   12 +
 .../GridCacheTransformEventSelfTest.java        |   70 +-
 ...niteCacheClientNodeChangingTopologyTest.java |   12 +-
 .../distributed/IgniteCacheGetRestartTest.java  |   29 +-
 .../IgniteCacheTxIteratorSelfTest.java          |   27 +-
 .../IgniteMvccTxTimeoutAbstractTest.java        |  146 +
 .../IgniteTxTimeoutAbstractTest.java            |   39 +-
 ...heColocatedMvccTxSingleThreadedSelfTest.java |   85 +
 .../dht/GridCacheDhtEntrySelfTest.java          |    5 +
 .../dht/GridCacheDhtMappingSelfTest.java        |    8 +
 .../GridCacheDhtPreloadDisabledSelfTest.java    |    8 +
 ...ridCacheDhtPreloadMultiThreadedSelfTest.java |    3 +
 .../dht/GridCacheDhtPreloadPutGetSelfTest.java  |    7 +
 .../dht/GridCacheDhtPreloadSelfTest.java        |    2 -
 .../IgniteCacheClearDuringRebalanceTest.java    |    9 +
 ...artitionedBackupNodeFailureRecoveryTest.java |    2 +-
 .../near/GridCacheNearClientHitTest.java        |    5 +-
 .../near/GridCacheNearMultiGetSelfTest.java     |   73 +-
 .../near/GridCacheNearMultiNodeSelfTest.java    |   10 +
 .../near/GridCacheNearOneNodeSelfTest.java      |   11 +-
 .../GridCacheNearPartitionedClearSelfTest.java  |    8 +
 .../GridCacheNearReaderPreloadSelfTest.java     |    8 +
 .../near/GridCacheNearReadersSelfTest.java      |   10 +-
 ...ePartitionedBasicStoreMultiNodeSelfTest.java |    6 +
 .../near/GridCachePartitionedEventSelfTest.java |    8 +
 ...titionedExplicitLockNodeFailureSelfTest.java |    8 +
 .../GridCachePartitionedLoadCacheSelfTest.java  |    8 +
 ...ePartitionedMvccTxMultiThreadedSelfTest.java |  101 +
 ...PartitionedMvccTxSingleThreadedSelfTest.java |   84 +
 ...idCachePartitionedMvccTxTimeoutSelfTest.java |   47 +
 ...achePartitionedPreloadLifecycleSelfTest.java |    1 -
 .../GridCachePartitionedTxTimeoutSelfTest.java  |   21 -
 .../near/GridNearCacheStoreUpdateTest.java      |    9 +
 .../near/NearCacheSyncUpdateTest.java           |   17 +
 .../near/NoneRebalanceModeSelfTest.java         |    9 +
 .../GridCacheReplicatedTxTimeoutSelfTest.java   |   18 -
 .../local/GridCacheLocalBasicApiSelfTest.java   |    8 +
 .../local/GridCacheLocalBasicStoreSelfTest.java |    8 +
 .../local/GridCacheLocalEventSelfTest.java      |    9 +
 .../GridCacheLocalEvictionEventSelfTest.java    |    8 +
 ...dCacheLocalGetAndTransformStoreSelfTest.java |    8 +
 .../GridCacheLocalIsolatedNodesSelfTest.java    |    8 +
 .../local/GridCacheLocalLoadAllSelfTest.java    |    8 +
 .../cache/local/GridCacheLocalLockSelfTest.java |    8 +
 .../GridCacheLocalMultithreadedSelfTest.java    |    8 +
 .../GridCacheLocalTxMultiThreadedSelfTest.java  |    9 +
 .../GridCacheLocalTxSingleThreadedSelfTest.java |    8 +
 .../local/GridCacheLocalTxTimeoutSelfTest.java  |    8 +
 ...vccAbstractBasicCoordinatorFailoverTest.java |  184 ++
 .../cache/mvcc/CacheMvccAbstractTest.java       |   19 +-
 ...eMvccPartitionedCoordinatorFailoverTest.java |   35 +
 .../cache/mvcc/CacheMvccTransactionsTest.java   |   28 +
 .../cache/mvcc/MvccUnsupportedTxModesTest.java  |  316 +++
 .../cache/persistence/DummyPageIO.java          |   21 +-
 ...eBaselineAffinityTopologyActivationTest.java |   59 -
 .../IgniteDataStorageMetricsSelfTest.java       |   17 +-
 ...tePdsBinaryMetadataOnClusterRestartTest.java |    4 +-
 .../persistence/IgnitePdsDynamicCacheTest.java  |    3 +-
 .../IgnitePdsExchangeDuringCheckpointTest.java  |    1 +
 .../persistence/IgnitePdsPageSizesTest.java     |    8 +
 .../IgnitePdsPartitionsStateRecoveryTest.java   |  170 ++
 ...lWalModeChangeDuringRebalancingSelfTest.java |   15 +
 .../db/IgnitePdsCacheRestoreTest.java           |    8 +-
 .../db/IgnitePdsDataRegionMetricsTest.java      |   10 +-
 .../db/IgnitePdsReserveWalSegmentsTest.java     |    2 -
 ...gniteCheckpointDirtyPagesForLowLoadTest.java |    1 -
 ...faultPageSizeBackwardsCompatibilityTest.java |   19 +-
 .../IgniteUidAsConsistentIdMigrationTest.java   |   26 +-
 .../persistence/db/wal/WalCompactionTest.java   |   26 +-
 .../db/wal/WalDeletionArchiveAbstractTest.java  |    1 -
 .../db/wal/reader/IgniteWalReaderTest.java      |    9 +-
 ...tinuousQueryAsyncFailoverMvccTxSelfTest.java |    5 +
 ...heContinuousQueryFailoverMvccTxSelfTest.java |    5 +
 ...teCacheContinuousQueryNoUnsubscribeTest.java |    5 -
 .../cache/transactions/TxRollbackAsyncTest.java |    6 +-
 .../processors/database/BPlusTreeSelfTest.java  |   23 +
 .../database/IgniteDbAbstractTest.java          |    2 -
 ...aStreamProcessorMvccPersistenceSelfTest.java |   28 +
 .../DataStreamProcessorPersistenceSelfTest.java |   28 +
 .../DataStreamProcessorSelfTest.java            |   63 +-
 .../internal/util/IgniteUtilsSelfTest.java      |   43 -
 .../loadtests/hashmap/GridCacheTestContext.java |    2 +
 ...mmunicationSpiConcurrentConnectSelfTest.java |    4 +-
 ...dTcpCommunicationSpiRecoveryAckSelfTest.java |    4 +-
 ...GridTcpCommunicationSpiRecoverySelfTest.java |    4 +-
 ...CommunicationRecoveryAckClosureSelfTest.java |    4 +-
 .../TcpDiscoveryPendingMessageDeliveryTest.java |    8 +-
 .../discovery/tcp/TcpDiscoveryRestartTest.java  |    8 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |    4 +-
 .../testframework/MvccFeatureChecker.java       |  137 +
 .../testframework/junits/GridAbstractTest.java  |   86 +-
 .../junits/common/GridAbstractExamplesTest.java |    5 +-
 .../junits/common/GridCommonAbstractTest.java   |    4 +-
 .../junits/spi/GridSpiAbstractTest.java         |    6 +-
 .../testsuites/IgniteBinaryCacheTestSuite.java  |    2 +
 .../IgniteCacheDataStructuresSelfTestSuite.java |    8 +
 .../testsuites/IgniteCacheMvccTestSuite.java    |    5 +
 .../testsuites/IgniteCacheMvccTestSuite2.java   |  197 ++
 .../ignite/testsuites/IgniteCacheTestSuite.java |    4 +
 .../testsuites/IgniteCacheTestSuite2.java       |  321 +--
 .../testsuites/IgniteCacheTestSuite6.java       |    3 -
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |    3 +
 .../util/mbeans/GridMBeanDisableSelfTest.java   |    4 +-
 modules/direct-io/pom.xml                       |   17 +-
 .../file/AlignedBuffersDirectFileIO.java        |  112 +-
 .../file/AlignedBuffersDirectFileIOFactory.java |   16 +-
 .../persistence/file/IgniteNativeIoLib.java     |    2 +-
 ...kPageCompressionIntegrationDirectIOTest.java |   31 +
 .../persistence/file/IgniteFileIOTest.java      |   15 +
 .../testsuites/IgnitePdsNativeIoTestSuite2.java |    4 +
 .../visor/verify/ValidateIndexesClosure.java    |    8 +-
 .../visor/verify/VisorValidateIndexesTask.java  |   29 +
 .../IgniteBinaryObjectFieldsQuerySelfTest.java  |    5 -
 .../cache/IgniteCacheAbstractQuerySelfTest.java |   54 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |    6 +-
 .../IgniteCacheStarvationOnRebalanceTest.java   |    5 +-
 .../mvcc/CacheMvccBackupsAbstractTest.java      |    2 +
 ...ccPartitionedSqlCoordinatorFailoverTest.java |   49 +-
 .../mvcc/CacheMvccSqlTxQueriesAbstractTest.java |    2 -
 .../db/wal/IgniteWalRecoveryTest.java           |    3 +-
 ...IgniteSqlSkipReducerOnUpdateDmlSelfTest.java |    5 -
 .../processors/query/SqlSchemaSelfTest.java     |   25 +-
 .../util/GridCommandHandlerIndexingTest.java    |    2 +-
 .../kafka/connect/IgniteSinkConnectorTest.java  |    2 -
 modules/ml/pom.xml                              |   13 +
 .../apache/ignite/ml/inference/InfModel.java    |   37 +
 .../ignite/ml/inference/ModelDescriptor.java    |   86 +
 .../ignite/ml/inference/ModelSignature.java     |   62 +
 .../inference/builder/AsyncInfModelBuilder.java |   43 +
 .../IgniteDistributedInfModelBuilder.java       |  367 +++
 .../builder/SingleInfModelBuilder.java          |   34 +
 .../inference/builder/SyncInfModelBuilder.java  |   42 +
 .../builder/ThreadedInfModelBuilder.java        |   86 +
 .../ml/inference/builder/package-info.java      |   22 +
 .../ignite/ml/inference/package-info.java       |   22 +
 .../parser/IgniteFunctionInfModelParser.java    |   76 +
 .../ml/inference/parser/InfModelParser.java     |   38 +
 .../parser/TensorFlowBaseInfModelParser.java    |  216 ++
 .../parser/TensorFlowGraphInfModelParser.java   |   40 +
 .../TensorFlowSavedModelInfModelParser.java     |   70 +
 .../ml/inference/parser/package-info.java       |   22 +
 .../reader/FileSystemInfModelReader.java        |   61 +
 .../reader/InMemoryInfModelReader.java          |   67 +
 .../ml/inference/reader/InfModelReader.java     |   33 +
 .../ml/inference/reader/package-info.java       |   22 +
 .../storage/IgniteModelDescriptorStorage.java   |   57 +
 .../storage/LocalModelDescriptorStorage.java    |   45 +
 .../storage/ModelDescriptorStorage.java         |   48 +
 .../ml/inference/storage/package-info.java      |   22 +
 .../ml/inference/util/DirectorySerializer.java  |  133 +
 .../ignite/ml/inference/util/package-info.java  |   22 +
 .../ignite/ml/multiclass/MultiClassModel.java   |  115 +
 .../ignite/ml/multiclass/OneVsRestTrainer.java  |  147 ++
 .../encoding/EncoderSortingStrategy.java        |   31 +
 .../preprocessing/encoding/EncoderTrainer.java  |   25 +-
 .../standardscaling/StandardScalerData.java     |   56 +
 .../StandardScalerPreprocessor.java             |   91 +
 .../standardscaling/StandardScalerTrainer.java  |  101 +
 .../standardscaling/package-info.java           |   22 +
 .../org/apache/ignite/ml/util/MnistUtils.java   |   79 +-
 .../org/apache/ignite/ml/IgniteMLTestSuite.java |    6 +-
 .../ignite/ml/inference/InferenceTestSuite.java |   38 +
 .../IgniteDistributedInfModelBuilderTest.java   |   71 +
 .../builder/InfModelBuilderTestUtil.java        |   53 +
 .../builder/SingleInfModelBuilderTest.java      |   42 +
 .../builder/ThreadedInfModelBuilderTest.java    |   44 +
 .../inference/util/DirectorySerializerTest.java |  126 +
 .../ml/multiclass/MultiClassTestSuite.java      |   32 +
 .../ml/multiclass/OneVsRestTrainerTest.java     |  126 +
 .../encoding/EncoderTrainerTest.java            |   27 +
 .../StandardScalerPreprocessorTest.java         |   59 +
 .../StandardScalerTrainerTest.java              |   85 +
 .../include/ignite/binary/binary_object.h       |    1 +
 .../include/ignite/binary/binary_raw_reader.h   |   20 +
 .../include/ignite/binary/binary_raw_writer.h   |   38 +-
 .../include/ignite/binary/binary_reader.h       |   26 +-
 .../include/ignite/binary/binary_writer.h       |   42 +-
 .../docs/source/pyignite.datatypes.base.rst     |    7 +
 .../python/docs/source/pyignite.datatypes.rst   |    1 +
 .../platforms/python/pyignite/datatypes/base.py |   24 +
 .../python/pyignite/datatypes/complex.py        |   23 +-
 .../python/pyignite/datatypes/internal.py       |   19 +-
 .../python/pyignite/datatypes/null_object.py    |    3 +-
 .../python/pyignite/datatypes/primitive.py      |    3 +-
 .../pyignite/datatypes/primitive_arrays.py      |    3 +-
 .../pyignite/datatypes/primitive_objects.py     |    3 +-
 .../python/pyignite/datatypes/standard.py       |    9 +-
 modules/platforms/python/pyignite/utils.py      |    6 +-
 modules/platforms/python/setup.py               |    2 +-
 .../platforms/python/tests/test_key_value.py    |   75 +-
 .../zk/internal/ZookeeperDiscoverySpiTest.java  |    4 +-
 parent/pom.xml                                  |    9 +-
 pom.xml                                         |    1 +
 419 files changed, 18474 insertions(+), 5165 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0063fb1d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
----------------------------------------------------------------------


[34/50] [abbrv] ignite git commit: IGNITE-8718: Fixed doxygen comments in C++

Posted by ag...@apache.org.
IGNITE-8718: Fixed doxygen comments in C++


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

Branch: refs/heads/ignite-9720
Commit: 9031733028ed79a35924319301279d3a2de8afbb
Parents: 1906002
Author: Igor Sapego <is...@apache.org>
Authored: Tue Nov 27 19:17:16 2018 +0300
Committer: Igor Sapego <is...@apache.org>
Committed: Tue Nov 27 19:17:16 2018 +0300

----------------------------------------------------------------------
 .../include/ignite/binary/binary_object.h       |  1 +
 .../include/ignite/binary/binary_raw_reader.h   | 20 ++++++++++
 .../include/ignite/binary/binary_raw_writer.h   | 38 +++++++++++++++++-
 .../include/ignite/binary/binary_reader.h       | 26 ++++++++++--
 .../include/ignite/binary/binary_writer.h       | 42 ++++++++++++++++++--
 5 files changed, 120 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/90317330/modules/platforms/cpp/binary/include/ignite/binary/binary_object.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/binary/binary_object.h b/modules/platforms/cpp/binary/include/ignite/binary/binary_object.h
index 4edbe84..bbad488 100644
--- a/modules/platforms/cpp/binary/include/ignite/binary/binary_object.h
+++ b/modules/platforms/cpp/binary/include/ignite/binary/binary_object.h
@@ -68,6 +68,7 @@ namespace ignite
              * @param mem Binary object memory.
              * @param start Object starting position in memory.
              * @param idRslvr ID resolver.
+             * @param metaMgr Metadata manager.
              */
             BinaryObject(impl::interop::InteropMemory& mem, int32_t start,
                 impl::binary::BinaryIdResolver* idRslvr, impl::binary::BinaryTypeManager* metaMgr) :

http://git-wip-us.apache.org/repos/asf/ignite/blob/90317330/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h b/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h
index 73d2525..dd7a696 100644
--- a/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h
+++ b/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h
@@ -342,6 +342,11 @@ namespace ignite
             /**
              * Start string array read.
              *
+             * Every time you get a BinaryStringArrayReader from BinaryRawReader
+             * you start reading session. Only one single reading session can be
+             * open at a time. So it is not allowed to start new reading session
+             * until all elements of the collection have been read.
+             *
              * @return String array reader.
              */
             BinaryStringArrayReader ReadStringArray();
@@ -349,6 +354,11 @@ namespace ignite
             /**
              * Start array read.
              *
+             * Every time you get a BinaryArrayReader from BinaryRawReader you
+             * start reading session. Only one single reading session can be
+             * open at a time. So it is not allowed to start new reading session
+             * until all elements of the collection have been read.
+             *
              * @return Array reader.
              */
             template<typename T>
@@ -364,6 +374,11 @@ namespace ignite
             /**
              * Start collection read.
              *
+             * Every time you get a BinaryCollectionReader from BinaryRawReader
+             * you start reading session. Only one single reading session can be
+             * open at a time. So it is not allowed to start new reading session
+             * until all elements of the collection have been read.
+             *
              * @return Collection reader.
              */
             template<typename T>
@@ -392,6 +407,11 @@ namespace ignite
             /**
              * Start map read.
              *
+             * Every time you get a BinaryMapReader from BinaryRawReader you
+             * start reading session. Only one single reading session can be
+             * open at a time. So it is not allowed to start new reading session
+             * until all elements of the collection have been read.
+             *
              * @return Map reader.
              */
             template<typename K, typename V>

http://git-wip-us.apache.org/repos/asf/ignite/blob/90317330/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_writer.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_writer.h b/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_writer.h
index 9dc73df..857d7f2 100644
--- a/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_writer.h
+++ b/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_writer.h
@@ -242,7 +242,7 @@ namespace ignite
              * @param val Array.
              * @param len Array length.
              */
-            void WriteTimeArray(const Time* val, const int32_t len);
+            void WriteTimeArray(const Time* val, int32_t len);
 
             /**
              * Write string.
@@ -272,6 +272,12 @@ namespace ignite
             /**
              * Start string array write.
              *
+             * Every time you get a BinaryStringArrayWriter from BinaryRawWriter
+             * you start writing session. Only one single writing session can be
+             * open at a time. So it is not allowed to start new writing session
+             * without calling BinaryStringArrayWriter::Close() method prior on
+             * obtained BinaryStringArrayWriter class instance.
+             *
              * @return String array writer.
              */
             BinaryStringArrayWriter WriteStringArray();
@@ -284,6 +290,12 @@ namespace ignite
             /**
              * Start array write.
              *
+             * Every time you get a BinaryArrayWriter from BinaryRawWriter you
+             * start writing session. Only one single writing session can be
+             * open at a time. So it is not allowed to start new writing session
+             * without calling BinaryArrayWriter::Close() method prior on
+             * obtained BinaryArrayWriter class instance.
+             *
              * @return Array writer.
              */
             template<typename T>
@@ -297,6 +309,12 @@ namespace ignite
             /**
              * Start collection write.
              *
+             * Every time you get a BinaryCollectionWriter from BinaryRawWriter
+             * you start writing session. Only one single writing session can be
+             * open at a time. So it is not allowed to start new writing session
+             * without calling BinaryCollectionWriter::Close() method prior on
+             * obtained BinaryCollectionWriter class instance.
+             *
              * @return Collection writer.
              */
             template<typename T>
@@ -308,6 +326,12 @@ namespace ignite
             /**
              * Start collection write.
              *
+             * Every time you get a BinaryCollectionWriter from BinaryRawWriter
+             * you start writing session. Only one single writing session can be
+             * open at a time. So it is not allowed to start new writing session
+             * without calling BinaryCollectionWriter::Close() method prior on
+             * obtained BinaryCollectionWriter class instance.
+             *
              * @param typ Collection type.
              * @return Collection writer.
              */
@@ -347,6 +371,12 @@ namespace ignite
             /**
              * Start map write.
              *
+             * Every time you get a BinaryMapWriter from BinaryRawWriter you
+             * start writing session. Only one single writing session can be
+             * open at a time. So it is not allowed to start new writing session
+             * without calling BinaryMapWriter::Close() method prior on obtained
+             * BinaryMapWriter class instance.
+             *
              * @return Map writer.
              */
             template<typename K, typename V>
@@ -358,6 +388,12 @@ namespace ignite
             /**
              * Start map write.
              *
+             * Every time you get a BinaryMapWriter from BinaryRawWriter you
+             * start writing session. Only one single writing session can be
+             * open at a time. So it is not allowed to start new writing session
+             * without calling BinaryMapWriter::Close() method prior on obtained
+             * BinaryMapWriter class instance.
+             *
              * @param typ Map type.
              * @return Map writer.
              */

http://git-wip-us.apache.org/repos/asf/ignite/blob/90317330/modules/platforms/cpp/binary/include/ignite/binary/binary_reader.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/binary/binary_reader.h b/modules/platforms/cpp/binary/include/ignite/binary/binary_reader.h
index 7103fec..acd9d61 100644
--- a/modules/platforms/cpp/binary/include/ignite/binary/binary_reader.h
+++ b/modules/platforms/cpp/binary/include/ignite/binary/binary_reader.h
@@ -272,7 +272,7 @@ namespace ignite
              *     to resulting array and returned value will contain required array length.
              *     -1 will be returned in case array in stream was null.
              */
-            int32_t ReadDateArray(const char* fieldName, Date* res, const int32_t len);
+            int32_t ReadDateArray(const char* fieldName, Date* res, int32_t len);
 
             /**
              * Read Timestamp. Maps to "Timestamp" type in Java.
@@ -293,7 +293,7 @@ namespace ignite
              *     to resulting array and returned value will contain required array length.
              *     -1 will be returned in case array in stream was null.
              */
-            int32_t ReadTimestampArray(const char* fieldName, Timestamp* res, const int32_t len);
+            int32_t ReadTimestampArray(const char* fieldName, Timestamp* res, int32_t len);
 
             /**
              * Read Time. Maps to "Time" type in Java.
@@ -314,7 +314,7 @@ namespace ignite
              *     to resulting array and returned value will contain required array length.
              *     -1 will be returned in case array in stream was null.
              */
-            int32_t ReadTimeArray(const char* fieldName, Time* res, const int32_t len);
+            int32_t ReadTimeArray(const char* fieldName, Time* res, int32_t len);
 
             /**
              * Read string.
@@ -355,6 +355,11 @@ namespace ignite
             /**
              * Start string array read.
              *
+             * Every time you get a BinaryStringArrayReader from BinaryReader
+             * you start reading session. Only one single reading session can be
+             * open at a time. So it is not allowed to start new reading session
+             * until all elements of the collection have been read.
+             *
              * @param fieldName Field name.
              * @return String array reader.
              */
@@ -363,6 +368,11 @@ namespace ignite
             /**
              * Start array read.
              *
+             * Every time you get a BinaryArrayReader from BinaryReader you
+             * start reading session. Only one single reading session can be
+             * open at a time. So it is not allowed to start new reading session
+             * until all elements of the collection have been read.
+             *
              * @param fieldName Field name.
              * @return Array reader.
              */
@@ -379,6 +389,11 @@ namespace ignite
             /**
              * Start collection read.
              *
+             * Every time you get a BinaryCollectionReader from BinaryReader you
+             * start reading session. Only one single reading session can be
+             * open at a time. So it is not allowed to start new reading session
+             * until all elements of the collection have been read.
+             *
              * @param fieldName Field name.
              * @return Collection reader.
              */
@@ -409,6 +424,11 @@ namespace ignite
             /**
              * Start map read.
              *
+             * Every time you get a BinaryMapReader from BinaryReader you start
+             * reading session. Only one single reading session can be open at
+             * a time. So it is not allowed to start new reading session until
+             * all elements of the collection have been read.
+             *
              * @param fieldName Field name.
              * @return Map reader.
              */

http://git-wip-us.apache.org/repos/asf/ignite/blob/90317330/modules/platforms/cpp/binary/include/ignite/binary/binary_writer.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/binary/binary_writer.h b/modules/platforms/cpp/binary/include/ignite/binary/binary_writer.h
index e609591..3dc1241 100644
--- a/modules/platforms/cpp/binary/include/ignite/binary/binary_writer.h
+++ b/modules/platforms/cpp/binary/include/ignite/binary/binary_writer.h
@@ -228,7 +228,7 @@ namespace ignite
              * @param val Array.
              * @param len Array length.
              */
-            void WriteDateArray(const char* fieldName, const Date* val, const int32_t len);
+            void WriteDateArray(const char* fieldName, const Date* val, int32_t len);
 
             /**
              * Write Timestamp. Maps to "Timestamp" type in Java.
@@ -245,7 +245,7 @@ namespace ignite
              * @param val Array.
              * @param len Array length.
              */
-            void WriteTimestampArray(const char* fieldName, const Timestamp* val, const int32_t len);
+            void WriteTimestampArray(const char* fieldName, const Timestamp* val, int32_t len);
 
             /**
              * Write Time. Maps to "Time" type in Java.
@@ -262,7 +262,7 @@ namespace ignite
              * @param val Array.
              * @param len Array length.
              */
-            void WriteTimeArray(const char* fieldName, const Time* val, const int32_t len);
+            void WriteTimeArray(const char* fieldName, const Time* val, int32_t len);
 
             /**
              * Write string.
@@ -295,6 +295,12 @@ namespace ignite
             /**
              * Start string array write.
              *
+             * Every time you get a BinaryStringArrayWriter from BinaryWriter
+             * you start writing session. Only one single writing session can be
+             * open at a time. So it is not allowed to start new writing session
+             * without calling BinaryStringArrayWriter::Close() method prior on
+             * obtained BinaryStringArrayWriter class instance.
+             *
              * @param fieldName Field name.
              * @return String array writer.
              */
@@ -310,6 +316,12 @@ namespace ignite
             /**
              * Start array write.
              *
+             * Every time you get a BinaryArrayWriter from BinaryWriter you
+             * start writing session. Only one single writing session can be
+             * open at a time. So it is not allowed to start new writing session
+             * without calling BinaryArrayWriter::Close() method prior on
+             * obtained BinaryArrayWriter class instance.
+             *
              * @param fieldName Field name.
              * @return Array writer.
              */
@@ -324,6 +336,12 @@ namespace ignite
             /**
              * Start collection write.
              *
+             * Every time you get a BinaryCollectionWriter from BinaryWriter you
+             * start writing session. Only one single writing session can be
+             * open at a time. So it is not allowed to start new writing session
+             * without calling BinaryCollectionWriter::Close() method prior on
+             * obtained BinaryCollectionWriter class instance.
+             *
              * @param fieldName Field name.
              * @return Collection writer.
              */
@@ -336,6 +354,12 @@ namespace ignite
             /**
              * Start collection write.
              *
+             * Every time you get a BinaryCollectionWriter from BinaryWriter you
+             * start writing session. Only one single writing session can be
+             * open at a time. So it is not allowed to start new writing session
+             * without calling BinaryCollectionWriter::Close() method prior on
+             * obtained BinaryCollectionWriter class instance.
+             *
              * @param fieldName Field name.
              * @param typ Collection type.
              * @return Collection writer.
@@ -378,6 +402,12 @@ namespace ignite
             /**
              * Start map write.
              *
+             * Every time you get a BinaryMapWriter from BinaryWriter you start
+             * writing session. Only one single writing session can be open at
+             * a time. So it is not allowed to start new writing session without
+             * calling BinaryMapWriter::Close() method prior on obtained
+             * BinaryMapWriter class instance.
+             *
              * @param fieldName Field name.
              * @return Map writer.
              */
@@ -390,6 +420,12 @@ namespace ignite
             /**
              * Start map write.
              *
+             * Every time you get a BinaryMapWriter from BinaryWriter you start
+             * writing session. Only one single writing session can be open at
+             * a time. So it is not allowed to start new writing session without
+             * calling BinaryMapWriter::Close() method prior on obtained
+             * BinaryMapWriter class instance.
+             *
              * @param fieldName Field name.
              * @param typ Map type.
              * @return Map writer.


[22/50] [abbrv] ignite git commit: IGNITE-9517: Replace uses of ConcurrentHashSet with GridConcurrentHashSet in tests. - Fixes #5397.

Posted by ag...@apache.org.
IGNITE-9517: Replace uses of ConcurrentHashSet with GridConcurrentHashSet in tests. - Fixes #5397.

Signed-off-by: Dmitriy Pavlov <dp...@apache.org>


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

Branch: refs/heads/ignite-9720
Commit: acc1024158ef600cdb078b0de8a9859beaa8b330
Parents: c870500
Author: shroman <rs...@yahoo.com>
Authored: Mon Nov 26 17:33:30 2018 +0300
Committer: Dmitriy Pavlov <dp...@apache.org>
Committed: Mon Nov 26 17:33:30 2018 +0300

----------------------------------------------------------------------
 .../internal/GridMultithreadedJobStealingSelfTest.java       | 6 +++---
 .../internal/binary/BinaryArrayIdentityResolverSelfTest.java | 4 ++--
 .../ignite/internal/binary/BinaryFieldsOffheapSelfTest.java  | 4 ++--
 .../internal/binary/BinaryFooterOffsetsOffheapSelfTest.java  | 4 ++--
 .../binary/BinarySerialiedFieldComparatorSelfTest.java       | 4 ++--
 ...GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java | 4 ++--
 .../processors/cache/distributed/CacheExchangeMergeTest.java | 4 ++--
 .../GridCachePartitionNotLoadedEventSelfTest.java            | 4 ++--
 .../cache/distributed/GridCacheTransformEventSelfTest.java   | 6 +++---
 .../IgniteCacheClientNodeChangingTopologyTest.java           | 4 ++--
 .../GridTcpCommunicationSpiConcurrentConnectSelfTest.java    | 4 ++--
 .../tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java      | 4 ++--
 .../tcp/GridTcpCommunicationSpiRecoverySelfTest.java         | 4 ++--
 .../IgniteTcpCommunicationRecoveryAckClosureSelfTest.java    | 4 ++--
 .../tcp/TcpDiscoveryPendingMessageDeliveryTest.java          | 8 ++++----
 .../ignite/spi/discovery/tcp/TcpDiscoveryRestartTest.java    | 8 ++++----
 .../ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java       | 4 ++--
 17 files changed, 40 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/acc10241/modules/core/src/test/java/org/apache/ignite/internal/GridMultithreadedJobStealingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridMultithreadedJobStealingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridMultithreadedJobStealingSelfTest.java
index 293c859..605cb3d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridMultithreadedJobStealingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridMultithreadedJobStealingSelfTest.java
@@ -37,6 +37,7 @@ import org.apache.ignite.compute.ComputeJobAdapter;
 import org.apache.ignite.compute.ComputeJobResult;
 import org.apache.ignite.compute.ComputeTaskAdapter;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.resources.LoggerResource;
@@ -45,7 +46,6 @@ import org.apache.ignite.spi.failover.jobstealing.JobStealingFailoverSpi;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.testframework.junits.common.GridCommonTest;
-import org.eclipse.jetty.util.ConcurrentHashSet;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -86,7 +86,7 @@ public class GridMultithreadedJobStealingSelfTest extends GridCommonAbstractTest
 
         final AtomicInteger stolen = new AtomicInteger(0);
         final AtomicInteger noneStolen = new AtomicInteger(0);
-        final ConcurrentHashSet nodes = new ConcurrentHashSet();
+        final GridConcurrentHashSet nodes = new GridConcurrentHashSet();
 
         int threadsNum = 10;
 
@@ -141,7 +141,7 @@ public class GridMultithreadedJobStealingSelfTest extends GridCommonAbstractTest
 
         final AtomicInteger stolen = new AtomicInteger(0);
         final AtomicInteger noneStolen = new AtomicInteger(0);
-        final ConcurrentHashSet nodes = new ConcurrentHashSet();
+        final GridConcurrentHashSet nodes = new GridConcurrentHashSet();
 
         int threadsNum = 10;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/acc10241/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryArrayIdentityResolverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryArrayIdentityResolverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryArrayIdentityResolverSelfTest.java
index 27c39c3..7548cf0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryArrayIdentityResolverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryArrayIdentityResolverSelfTest.java
@@ -26,10 +26,10 @@ import org.apache.ignite.binary.BinaryWriter;
 import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.eclipse.jetty.util.ConcurrentHashSet;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -42,7 +42,7 @@ import static org.junit.Assert.assertNotEquals;
  */
 public class BinaryArrayIdentityResolverSelfTest extends GridCommonAbstractTest {
     /** Pointers to release. */
-    private final Set<Long> ptrs = new ConcurrentHashSet<>();
+    private final Set<Long> ptrs = new GridConcurrentHashSet<>();
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {

http://git-wip-us.apache.org/repos/asf/ignite/blob/acc10241/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldsOffheapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldsOffheapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldsOffheapSelfTest.java
index ca4bdd2..e5b3648 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldsOffheapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldsOffheapSelfTest.java
@@ -17,15 +17,15 @@
 
 package org.apache.ignite.internal.binary;
 
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.GridUnsafe;
-import org.eclipse.jetty.util.ConcurrentHashSet;
 
 /**
  * Field tests for heap-based binaries.
  */
 public class BinaryFieldsOffheapSelfTest extends BinaryFieldsAbstractSelfTest {
     /** Allocated unsafe pointer. */
-    private final ConcurrentHashSet<Long> ptrs = new ConcurrentHashSet<>();
+    private final GridConcurrentHashSet<Long> ptrs = new GridConcurrentHashSet<>();
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {

http://git-wip-us.apache.org/repos/asf/ignite/blob/acc10241/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFooterOffsetsOffheapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFooterOffsetsOffheapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFooterOffsetsOffheapSelfTest.java
index 052e652..8e00457 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFooterOffsetsOffheapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFooterOffsetsOffheapSelfTest.java
@@ -17,15 +17,15 @@
 
 package org.apache.ignite.internal.binary;
 
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.GridUnsafe;
-import org.eclipse.jetty.util.ConcurrentHashSet;
 
 /**
  * Compact offsets tests for offheap binary objects.
  */
 public class BinaryFooterOffsetsOffheapSelfTest extends BinaryFooterOffsetsAbstractSelfTest {
     /** Allocated unsafe pointer. */
-    private final ConcurrentHashSet<Long> ptrs = new ConcurrentHashSet<>();
+    private final GridConcurrentHashSet<Long> ptrs = new GridConcurrentHashSet<>();
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {

http://git-wip-us.apache.org/repos/asf/ignite/blob/acc10241/modules/core/src/test/java/org/apache/ignite/internal/binary/BinarySerialiedFieldComparatorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinarySerialiedFieldComparatorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinarySerialiedFieldComparatorSelfTest.java
index 4278ef4..1e0c5c9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinarySerialiedFieldComparatorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinarySerialiedFieldComparatorSelfTest.java
@@ -19,10 +19,10 @@ package org.apache.ignite.internal.binary;
 
 import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.eclipse.jetty.util.ConcurrentHashSet;
 
 import java.math.BigDecimal;
 import java.sql.Timestamp;
@@ -42,7 +42,7 @@ public class BinarySerialiedFieldComparatorSelfTest extends GridCommonAbstractTe
     private static final String FIELD_SINGLE = "single";
 
     /** Pointers to release. */
-    private final Set<Long> ptrs = new ConcurrentHashSet<>();
+    private final Set<Long> ptrs = new GridConcurrentHashSet<>();
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {

http://git-wip-us.apache.org/repos/asf/ignite/blob/acc10241/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
index 81614cb..3da8605 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
@@ -31,6 +31,7 @@ import org.apache.ignite.IgniteBinary;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
@@ -41,7 +42,6 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.eclipse.jetty.util.ConcurrentHashSet;
 
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 
@@ -91,7 +91,7 @@ public class GridCacheClientNodeBinaryObjectMetadataMultinodeTest extends GridCo
 
         final AtomicBoolean stop = new AtomicBoolean();
 
-        final ConcurrentHashSet<String> allTypes = new ConcurrentHashSet<>();
+        final GridConcurrentHashSet<String> allTypes = new GridConcurrentHashSet<>();
 
         IgniteInternalFuture<?> fut;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/acc10241/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheExchangeMergeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheExchangeMergeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheExchangeMergeTest.java
index 93c1066..03ea539 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheExchangeMergeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheExchangeMergeTest.java
@@ -58,6 +58,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleRequest;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
@@ -74,7 +75,6 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
-import org.eclipse.jetty.util.ConcurrentHashSet;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_EXCHANGE_HISTORY_SIZE;
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
@@ -354,7 +354,7 @@ public class CacheExchangeMergeTest extends GridCommonAbstractTest {
 
             final AtomicInteger idx = new AtomicInteger(initNodes);
 
-            final ConcurrentHashSet<Integer> stopNodes = new ConcurrentHashSet<>();
+            final GridConcurrentHashSet<Integer> stopNodes = new GridConcurrentHashSet<>();
 
             IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
                 @Override public Void call() throws Exception {

http://git-wip-us.apache.org/repos/asf/ignite/blob/acc10241/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
index 92783b0..334608d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
@@ -32,6 +32,7 @@ import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
@@ -41,7 +42,6 @@ import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.GridTestUtils.SF;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.util.TestTcpCommunicationSpi;
-import org.eclipse.jetty.util.ConcurrentHashSet;
 
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
@@ -277,7 +277,7 @@ public class GridCachePartitionNotLoadedEventSelfTest extends GridCommonAbstract
      */
     private static class PartitionNotFullyLoadedListener implements IgnitePredicate<Event> {
         /** */
-        private Collection<Integer> lostParts = new ConcurrentHashSet<>();
+        private Collection<Integer> lostParts = new GridConcurrentHashSet<>();
 
         /** {@inheritDoc} */
         @Override public boolean apply(Event evt) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/acc10241/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java
index b1701a4..913321e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java
@@ -36,6 +36,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.TransactionConfiguration;
 import org.apache.ignite.events.CacheEvent;
 import org.apache.ignite.events.Event;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.resources.IgniteInstanceResource;
@@ -45,7 +46,6 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
-import org.eclipse.jetty.util.ConcurrentHashSet;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -96,7 +96,7 @@ public class GridCacheTransformEventSelfTest extends GridCommonAbstractTest {
     private IgniteCache<Integer, Integer>[] caches;
 
     /** Recorded events. */
-    private ConcurrentHashSet<CacheEvent> evts;
+    private GridConcurrentHashSet<CacheEvent> evts;
 
     /** Cache mode. */
     private CacheMode cacheMode;
@@ -174,7 +174,7 @@ public class GridCacheTransformEventSelfTest extends GridCommonAbstractTest {
         this.txConcurrency = txConcurrency;
         this.txIsolation = txIsolation;
 
-        evts = new ConcurrentHashSet<>();
+        evts = new GridConcurrentHashSet<>();
 
         startGridsMultiThreaded(GRID_CNT, true);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/acc10241/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
index 300ecb9..ce8749c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
@@ -68,6 +68,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCach
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockRequest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
@@ -89,7 +90,6 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
-import org.eclipse.jetty.util.ConcurrentHashSet;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
@@ -1802,7 +1802,7 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
 
         final int THREADS = CLIENT_CNT * 3;
 
-        final ConcurrentHashSet<Integer> putKeys = new ConcurrentHashSet<>();
+        final GridConcurrentHashSet<Integer> putKeys = new GridConcurrentHashSet<>();
 
         IgniteInternalFuture<?> fut;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/acc10241/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
index ce96c55..9013016 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
@@ -39,6 +39,7 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteNodeAttributes;
 import org.apache.ignite.internal.managers.communication.GridIoMessageFactory;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.nio.GridCommunicationClient;
 import org.apache.ignite.internal.util.nio.GridNioServer;
@@ -58,7 +59,6 @@ import org.apache.ignite.testframework.junits.IgniteMock;
 import org.apache.ignite.testframework.junits.IgniteTestResources;
 import org.apache.ignite.testframework.junits.spi.GridSpiAbstractTest;
 import org.apache.ignite.testframework.junits.spi.GridSpiTest;
-import org.eclipse.jetty.util.ConcurrentHashSet;
 
 /**
  *
@@ -125,7 +125,7 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
         private final AtomicInteger cntr = new AtomicInteger();
 
         /** */
-        private final ConcurrentHashSet<Long> msgIds = new ConcurrentHashSet<>();
+        private final GridConcurrentHashSet<Long> msgIds = new GridConcurrentHashSet<>();
 
         /**
          * @param latch Latch.

http://git-wip-us.apache.org/repos/asf/ignite/blob/acc10241/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
index e8a535b..16ad2c8 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
@@ -29,6 +29,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.managers.communication.GridIoMessageFactory;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.nio.GridNioRecoveryDescriptor;
 import org.apache.ignite.internal.util.nio.GridNioServer;
@@ -49,7 +50,6 @@ import org.apache.ignite.testframework.junits.GridTestKernalContext;
 import org.apache.ignite.testframework.junits.IgniteTestResources;
 import org.apache.ignite.testframework.junits.spi.GridSpiAbstractTest;
 import org.apache.ignite.testframework.junits.spi.GridSpiTest;
-import org.eclipse.jetty.util.ConcurrentHashSet;
 
 /**
  *
@@ -92,7 +92,7 @@ public class GridTcpCommunicationSpiRecoveryAckSelfTest<T extends CommunicationS
     /** */
     private class TestListener implements CommunicationListener<Message> {
         /** */
-        private ConcurrentHashSet<Long> msgIds = new ConcurrentHashSet<>();
+        private GridConcurrentHashSet<Long> msgIds = new GridConcurrentHashSet<>();
 
         /** */
         private AtomicInteger rcvCnt = new AtomicInteger();

http://git-wip-us.apache.org/repos/asf/ignite/blob/acc10241/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
index 5d09fbb..12cf4d9 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
@@ -34,6 +34,7 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.managers.communication.GridIoMessageFactory;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.nio.GridNioServer;
 import org.apache.ignite.internal.util.nio.GridNioSession;
@@ -54,7 +55,6 @@ import org.apache.ignite.testframework.junits.IgniteMock;
 import org.apache.ignite.testframework.junits.IgniteTestResources;
 import org.apache.ignite.testframework.junits.spi.GridSpiAbstractTest;
 import org.apache.ignite.testframework.junits.spi.GridSpiTest;
-import org.eclipse.jetty.util.ConcurrentHashSet;
 
 /**
  *
@@ -114,7 +114,7 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
         private CountDownLatch blockLatch;
 
         /** */
-        private ConcurrentHashSet<Long> msgIds = new ConcurrentHashSet<>();
+        private GridConcurrentHashSet<Long> msgIds = new GridConcurrentHashSet<>();
 
         /** */
         private AtomicInteger rcvCnt = new AtomicInteger();

http://git-wip-us.apache.org/repos/asf/ignite/blob/acc10241/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java
index 8f8d101..5bee756 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java
@@ -30,6 +30,7 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.managers.communication.GridIoMessageFactory;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.nio.GridNioRecoveryDescriptor;
 import org.apache.ignite.internal.util.nio.GridNioServer;
@@ -52,7 +53,6 @@ import org.apache.ignite.testframework.junits.GridTestKernalContext;
 import org.apache.ignite.testframework.junits.IgniteTestResources;
 import org.apache.ignite.testframework.junits.spi.GridSpiAbstractTest;
 import org.apache.ignite.testframework.junits.spi.GridSpiTest;
-import org.eclipse.jetty.util.ConcurrentHashSet;
 
 /**
  *
@@ -96,7 +96,7 @@ public class IgniteTcpCommunicationRecoveryAckClosureSelfTest<T extends Communic
     /** */
     private class TestListener implements CommunicationListener<Message> {
         /** */
-        private ConcurrentHashSet<Long> msgIds = new ConcurrentHashSet<>();
+        private GridConcurrentHashSet<Long> msgIds = new GridConcurrentHashSet<>();
 
         /** */
         private AtomicInteger rcvCnt = new AtomicInteger();

http://git-wip-us.apache.org/repos/asf/ignite/blob/acc10241/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryPendingMessageDeliveryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryPendingMessageDeliveryTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryPendingMessageDeliveryTest.java
index 9b3dfee..5c1b5fe 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryPendingMessageDeliveryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryPendingMessageDeliveryTest.java
@@ -29,13 +29,13 @@ import org.apache.ignite.internal.managers.discovery.DiscoCache;
 import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAbstractMessage;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.eclipse.jetty.util.ConcurrentHashSet;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -54,7 +54,7 @@ public class TcpDiscoveryPendingMessageDeliveryTest extends GridCommonAbstractTe
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         blockMsgs = false;
-        receivedEnsuredMsgs = new ConcurrentHashSet<>();
+        receivedEnsuredMsgs = new GridConcurrentHashSet<>();
     }
 
     /** {@inheritDoc} */
@@ -88,7 +88,7 @@ public class TcpDiscoveryPendingMessageDeliveryTest extends GridCommonAbstractTe
         Ignite coord = startGrid("coordinator");
         TcpDiscoverySpi coordDisco = (TcpDiscoverySpi)coord.configuration().getDiscoverySpi();
 
-        Set<TcpDiscoveryAbstractMessage> sentEnsuredMsgs = new ConcurrentHashSet<>();
+        Set<TcpDiscoveryAbstractMessage> sentEnsuredMsgs = new GridConcurrentHashSet<>();
         coordDisco.addSendMessageListener(msg -> {
             if (coordDisco.ensured(msg))
                 sentEnsuredMsgs.add(msg);
@@ -143,7 +143,7 @@ public class TcpDiscoveryPendingMessageDeliveryTest extends GridCommonAbstractTe
         Ignite coord = startGrid("coordinator");
         TcpDiscoverySpi coordDisco = (TcpDiscoverySpi)coord.configuration().getDiscoverySpi();
 
-        Set<TcpDiscoveryAbstractMessage> sentEnsuredMsgs = new ConcurrentHashSet<>();
+        Set<TcpDiscoveryAbstractMessage> sentEnsuredMsgs = new GridConcurrentHashSet<>();
         coordDisco.addSendMessageListener(msg -> {
             if (coordDisco.ensured(msg))
                 sentEnsuredMsgs.add(msg);

http://git-wip-us.apache.org/repos/asf/ignite/blob/acc10241/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryRestartTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryRestartTest.java
index cacefa5..96ac260 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryRestartTest.java
@@ -30,13 +30,13 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.eclipse.jetty.util.ConcurrentHashSet;
 
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
@@ -90,7 +90,7 @@ public class TcpDiscoveryRestartTest extends GridCommonAbstractTest {
 
         startGrids(NODE_CNT);
 
-        final ConcurrentHashSet<UUID> nodeIds = new ConcurrentHashSet<>();
+        final GridConcurrentHashSet<UUID> nodeIds = new GridConcurrentHashSet<>();
 
         final AtomicInteger id = new AtomicInteger(NODE_CNT);
 
@@ -172,10 +172,10 @@ public class TcpDiscoveryRestartTest extends GridCommonAbstractTest {
      */
     private class TestEventListener implements IgnitePredicate<Event> {
         /** */
-        private final ConcurrentHashSet<UUID> joinIds = new ConcurrentHashSet<>();
+        private final GridConcurrentHashSet<UUID> joinIds = new GridConcurrentHashSet<>();
 
         /** */
-        private final ConcurrentHashSet<UUID> leftIds = new ConcurrentHashSet<>();
+        private final GridConcurrentHashSet<UUID> leftIds = new GridConcurrentHashSet<>();
 
         /** {@inheritDoc} */
         @Override public boolean apply(Event evt) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/acc10241/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
index 1aae8fb..32df795 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
@@ -58,6 +58,7 @@ import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
 import org.apache.ignite.internal.processors.continuous.StartRoutineAckDiscoveryMessage;
 import org.apache.ignite.internal.processors.port.GridPortRecord;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
@@ -89,7 +90,6 @@ import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeLeftMessage;
 import org.apache.ignite.testframework.GridStringLogger;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.eclipse.jetty.util.ConcurrentHashSet;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
@@ -2372,7 +2372,7 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
      */
     private static class TestEventDiscardSpi extends TcpDiscoverySpi {
         /** */
-        private ConcurrentHashSet<IgniteUuid> msgIds = new ConcurrentHashSet<>();
+        private GridConcurrentHashSet<IgniteUuid> msgIds = new GridConcurrentHashSet<>();
 
         /** */
         private volatile boolean checkDuplicates;


[50/50] [abbrv] ignite git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-9720

Posted by ag...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-9720


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

Branch: refs/heads/ignite-9720
Commit: 04fdbc1afd989677991bca876fb0c2d7862262a1
Parents: 0063fb1 fe8c8cc
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Wed Nov 28 14:45:02 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Nov 28 14:45:02 2018 +0300

----------------------------------------------------------------------
 examples/sql/world.sql                          |  28 +--
 .../GridCacheDatabaseSharedManager.java         |  20 ++-
 .../cache/CacheEnumOperationsAbstractTest.java  |   2 -
 ...PdsCacheStartStopWithFreqCheckpointTest.java | 177 +++++++++++++++++++
 ...GridTcpCommunicationSpiRecoverySelfTest.java |   1 -
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |   3 +
 modules/mesos/pom.xml                           |  48 -----
 .../ignite/mesos/resource/IgniteProvider.java   |   1 -
 .../components/modal-import-models/service.js   |  24 ++-
 9 files changed, 230 insertions(+), 74 deletions(-)
----------------------------------------------------------------------



[33/50] [abbrv] ignite git commit: IGNITE-7441 Drop IGNITE_SERVICES_COMPATIBILITY_MODE system property - Fixes #5482.

Posted by ag...@apache.org.
IGNITE-7441 Drop IGNITE_SERVICES_COMPATIBILITY_MODE system property - Fixes #5482.

Signed-off-by: Ilya Kasnacheev <il...@gmail.com>


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

Branch: refs/heads/ignite-9720
Commit: 19060028185a82878891768ec34adc553384ef5e
Parents: 41f4225
Author: Vyacheslav Daradur <da...@gmail.com>
Authored: Tue Nov 27 18:45:28 2018 +0300
Committer: Ilya Kasnacheev <il...@gmail.com>
Committed: Tue Nov 27 18:45:28 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   | 15 -------
 .../ignite/internal/IgniteNodeAttributes.java   |  3 --
 .../discovery/GridDiscoveryManager.java         | 17 --------
 .../service/GridServiceProcessor.java           | 12 ------
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 41 --------------------
 .../GridDiscoveryManagerAttributesSelfTest.java | 37 ------------------
 6 files changed, 125 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/19060028/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 3c59ea1..d581e75 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -591,21 +591,6 @@ public final class IgniteSystemProperties {
         "IGNITE_H2_INDEXING_CACHE_THREAD_USAGE_TIMEOUT";
 
     /**
-     * Manages backward compatibility of {@link IgniteServices}. All nodes in cluster must have identical value
-     * of this property.
-     * <p>
-     * If property is {@code false} then node is not required to have service implementation class if service is not
-     * deployed on this node.
-     * <p>
-     * If the property is {@code true} then service implementation class is required on node even if service
-     * is not deployed on this node.
-     * <p>
-     * If the property is not set ({@code null}) then Ignite will automatically detect which compatibility mode
-     * should be used.
-     */
-    public static final String IGNITE_SERVICES_COMPATIBILITY_MODE = "IGNITE_SERVICES_COMPATIBILITY_MODE";
-
-    /**
      * Manages backward compatibility of {@link StreamTransformer#from(CacheEntryProcessor)} method.
      * <p>
      * If the property is {@code true}, then the wrapped {@link CacheEntryProcessor} won't be able to be loaded over

http://git-wip-us.apache.org/repos/asf/ignite/blob/19060028/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
index 4ca4f1b..24a71a8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
@@ -174,9 +174,6 @@ public final class IgniteNodeAttributes {
     /** Late affinity assignment mode. */
     public static final String ATTR_LATE_AFFINITY_ASSIGNMENT = ATTR_PREFIX + ".cache.lateAffinity";
 
-    /** Ignite services compatibility mode (can be {@code null}). */
-    public static final String ATTR_SERVICES_COMPATIBILITY_MODE = ATTR_PREFIX + ".services.compatibility.enabled";
-
     /** Late affinity assignment mode. */
     public static final String ATTR_ACTIVE_ON_START = ATTR_PREFIX + ".active.on.start";
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/19060028/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 0a982b0..9a0ca92 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -144,7 +144,6 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISCOVERY_HISTORY_SIZE;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SECURITY_COMPATIBILITY_MODE;
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_SERVICES_COMPATIBILITY_MODE;
 import static org.apache.ignite.IgniteSystemProperties.getInteger;
 import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED;
 import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_RECONNECTED;
@@ -165,7 +164,6 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_OFFHEAP_SIZE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PEER_CLASSLOADING;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PHY_RAM;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_COMPATIBILITY_MODE;
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SERVICES_COMPATIBILITY_MODE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_USER_NAME;
 import static org.apache.ignite.internal.IgniteVersionUtils.VER;
 import static org.apache.ignite.internal.events.DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT;
@@ -1219,7 +1217,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
         boolean locDelayAssign = locNode.attribute(ATTR_LATE_AFFINITY_ASSIGNMENT);
 
-        Boolean locSrvcCompatibilityEnabled = locNode.attribute(ATTR_SERVICES_COMPATIBILITY_MODE);
         Boolean locSecurityCompatibilityEnabled = locNode.attribute(ATTR_SECURITY_COMPATIBILITY_MODE);
 
         for (ClusterNode n : nodes) {
@@ -1305,20 +1302,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     ", rmtAddrs=" + U.addressesAsString(n) + ", rmtNode=" + U.toShortString(n) + "]");
             }
 
-            Boolean rmtSrvcCompatibilityEnabled = n.attribute(ATTR_SERVICES_COMPATIBILITY_MODE);
-
-            if (!F.eq(locSrvcCompatibilityEnabled, rmtSrvcCompatibilityEnabled)) {
-                throw new IgniteCheckedException("Local node's " + IGNITE_SERVICES_COMPATIBILITY_MODE +
-                    " property value differs from remote node's value " +
-                    "(to make sure all nodes in topology have identical IgniteServices compatibility mode enabled, " +
-                    "configure system property explicitly) " +
-                    "[locSrvcCompatibilityEnabled=" + locSrvcCompatibilityEnabled +
-                    ", rmtSrvcCompatibilityEnabled=" + rmtSrvcCompatibilityEnabled +
-                    ", locNodeAddrs=" + U.addressesAsString(locNode) +
-                    ", rmtNodeAddrs=" + U.addressesAsString(n) +
-                    ", locNodeId=" + locNode.id() + ", rmtNode=" + U.toShortString(n) + "]");
-            }
-
             if (n.version().compareToIgnoreTimestamp(SERVICE_PERMISSIONS_SINCE) >= 0
                 && ctx.security().enabled() // Matters only if security enabled.
                ) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/19060028/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index 271204c..d4ec78f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -110,11 +110,8 @@ import org.apache.ignite.transactions.Transaction;
 import org.jetbrains.annotations.Nullable;
 
 import static javax.cache.event.EventType.REMOVED;
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_SERVICES_COMPATIBILITY_MODE;
-import static org.apache.ignite.IgniteSystemProperties.getString;
 import static org.apache.ignite.configuration.DeploymentMode.ISOLATED;
 import static org.apache.ignite.configuration.DeploymentMode.PRIVATE;
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SERVICES_COMPATIBILITY_MODE;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
@@ -124,9 +121,6 @@ import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_REA
  */
 @SuppressWarnings({"SynchronizationOnLocalVariableOrMethodParameter", "ConstantConditions"})
 public class GridServiceProcessor extends GridProcessorAdapter implements IgniteChangeGlobalStateSupport {
-    /** */
-    private final Boolean srvcCompatibilitySysProp;
-
     /** Time to wait before reassignment retries. */
     private static final long RETRY_TIMEOUT = 1000;
 
@@ -183,10 +177,6 @@ public class GridServiceProcessor extends GridProcessorAdapter implements Ignite
 
         depExe = Executors.newSingleThreadExecutor(new IgniteThreadFactory(ctx.igniteInstanceName(),
             "srvc-deploy", oomeHnd));
-
-        String servicesCompatibilityMode = getString(IGNITE_SERVICES_COMPATIBILITY_MODE);
-
-        srvcCompatibilitySysProp = servicesCompatibilityMode == null ? null : Boolean.valueOf(servicesCompatibilityMode);
     }
 
     /**
@@ -205,8 +195,6 @@ public class GridServiceProcessor extends GridProcessorAdapter implements Ignite
 
     /** {@inheritDoc} */
     @Override public void start() throws IgniteCheckedException {
-        ctx.addNodeAttribute(ATTR_SERVICES_COMPATIBILITY_MODE, srvcCompatibilitySysProp);
-
         if (ctx.isDaemon())
             return;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/19060028/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 2c3569e..4c74706 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -150,7 +150,6 @@ import org.jetbrains.annotations.Nullable;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISCOVERY_CLIENT_RECONNECT_HISTORY_SIZE;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID;
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_SERVICES_COMPATIBILITY_MODE;
 import static org.apache.ignite.IgniteSystemProperties.getInteger;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
@@ -164,7 +163,6 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_COMPACT_FOOTER;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_BINARY_STRING_SER_VER_2;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_DFLT_SUID;
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SERVICES_COMPATIBILITY_MODE;
 import static org.apache.ignite.spi.IgnitePortProtocol.TCP;
 import static org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoverySpiState.AUTH_FAILED;
 import static org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoverySpiState.CHECK_FAILED;
@@ -4030,45 +4028,6 @@ class ServerImpl extends TcpDiscoveryImpl {
                     return;
                 }
 
-                final Boolean locSrvcCompatibilityEnabled = locNode.attribute(ATTR_SERVICES_COMPATIBILITY_MODE);
-
-                final Boolean rmtSrvcCompatibilityEnabled = node.attribute(ATTR_SERVICES_COMPATIBILITY_MODE);
-
-                if (!F.eq(locSrvcCompatibilityEnabled, rmtSrvcCompatibilityEnabled)) {
-                    utilityPool.execute(
-                        new Runnable() {
-                            @Override public void run() {
-                                String errMsg = "Local node's " + IGNITE_SERVICES_COMPATIBILITY_MODE +
-                                    " property value differs from remote node's value " +
-                                    "(to make sure all nodes in topology have identical IgniteServices compatibility mode, " +
-                                    "configure system property explicitly) " +
-                                    "[locSrvcCompatibilityEnabled=" + locSrvcCompatibilityEnabled +
-                                    ", rmtSrvcCompatibilityEnabled=" + rmtSrvcCompatibilityEnabled +
-                                    ", locNodeAddrs=" + U.addressesAsString(locNode) +
-                                    ", rmtNodeAddrs=" + U.addressesAsString(node) +
-                                    ", locNodeId=" + locNode.id() + ", rmtNodeId=" + msg.creatorNodeId() + ']';
-
-                                String sndMsg = "Local node's " + IGNITE_SERVICES_COMPATIBILITY_MODE +
-                                    " property value differs from remote node's value " +
-                                    "(to make sure all nodes in topology have identical IgniteServices compatibility mode, " +
-                                    "configure system property explicitly) " +
-                                    "[locSrvcCompatibilityEnabled=" + rmtSrvcCompatibilityEnabled +
-                                    ", rmtSrvcCompatibilityEnabled=" + locSrvcCompatibilityEnabled +
-                                    ", locNodeAddrs=" + U.addressesAsString(node) + ", locPort=" + node.discoveryPort() +
-                                    ", rmtNodeAddr=" + U.addressesAsString(locNode) + ", locNodeId=" + node.id() +
-                                    ", rmtNodeId=" + locNode.id() + ']';
-
-                                nodeCheckError(
-                                    node,
-                                    errMsg,
-                                    sndMsg);
-                            }
-                        });
-
-                    // Ignore join request.
-                    return;
-                }
-
                 // Handle join.
                 node.internalOrder(ring.nextNodeOrder());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/19060028/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
index 69f95e8..d19ae72 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
@@ -34,7 +34,6 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SECURITY_COMPATIBILITY_MODE;
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_SERVICES_COMPATIBILITY_MODE;
 import static org.apache.ignite.configuration.DeploymentMode.CONTINUOUS;
 import static org.apache.ignite.configuration.DeploymentMode.SHARED;
 
@@ -248,42 +247,6 @@ public abstract class GridDiscoveryManagerAttributesSelfTest extends GridCommonA
     /**
      * @throws Exception If failed.
      */
-    public void testServiceCompatibilityEnabled() throws Exception {
-        String backup = System.getProperty(IGNITE_SERVICES_COMPATIBILITY_MODE);
-
-        try {
-            doTestServiceCompatibilityEnabled(true, null, true);
-            doTestServiceCompatibilityEnabled(false, null, true);
-            doTestServiceCompatibilityEnabled(null, false, true);
-            doTestServiceCompatibilityEnabled(true, false, true);
-            doTestServiceCompatibilityEnabled(null, true, true);
-            doTestServiceCompatibilityEnabled(false, true, true);
-
-            doTestServiceCompatibilityEnabled(true, true, false);
-            doTestServiceCompatibilityEnabled(false, false, false);
-            doTestServiceCompatibilityEnabled(null, null, false);
-        }
-        finally {
-            if (backup != null)
-                System.setProperty(IGNITE_SERVICES_COMPATIBILITY_MODE, backup);
-            else
-                System.clearProperty(IGNITE_SERVICES_COMPATIBILITY_MODE);
-        }
-    }
-
-    /**
-     * @param first Service compatibility enabled flag for first node.
-     * @param second Service compatibility enabled flag for second node.
-     * @param fail Fail flag.
-     * @throws Exception If failed.
-     */
-    private void doTestServiceCompatibilityEnabled(Object first, Object second, boolean fail) throws Exception {
-        doTestCompatibilityEnabled(IGNITE_SERVICES_COMPATIBILITY_MODE, first, second, fail);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     public void testSecurityCompatibilityEnabled() throws Exception {
         TestReconnectPluginProvider.enabled = true;
         TestReconnectProcessor.enabled = true;


[49/50] [abbrv] ignite git commit: IGNITE-10298 Cover possible deadlock in case of caches start and frequent checkpoints. - Fixes #5517.

Posted by ag...@apache.org.
IGNITE-10298 Cover possible deadlock in case of caches start and frequent checkpoints. - Fixes #5517.

Signed-off-by: Pavel Kovalenko <jo...@gmail.com>


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

Branch: refs/heads/ignite-9720
Commit: fe8c8cc5816872ba58bbd2401698b9e847788ea0
Parents: 2418d87
Author: Pavel Kovalenko <jo...@gmail.com>
Authored: Wed Nov 28 13:32:50 2018 +0300
Committer: Pavel Kovalenko <jo...@gmail.com>
Committed: Wed Nov 28 13:32:50 2018 +0300

----------------------------------------------------------------------
 .../GridCacheDatabaseSharedManager.java         |  20 ++-
 ...PdsCacheStartStopWithFreqCheckpointTest.java | 177 +++++++++++++++++++
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |   3 +
 3 files changed, 198 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fe8c8cc5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 9a083f8..eb52ee6 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -250,6 +250,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     /** Checkpoint thread. Needs to be volatile because it is created in exchange worker. */
     private volatile Checkpointer checkpointer;
 
+    /** Checkpointer thread instance. */
+    private volatile IgniteThread checkpointerThread;
+
     /** For testing only. */
     private volatile boolean checkpointsEnabled = true;
 
@@ -427,6 +430,15 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
     /**
      * For test use only.
+     *
+     * @return Checkpointer thread instance.
+     */
+    public IgniteThread checkpointerThread() {
+        return checkpointerThread;
+    }
+
+    /**
+     * For test use only.
      */
     public IgniteInternalFuture<Void> enableCheckpoints(boolean enable) {
         GridFutureAdapter<Void> fut = new GridFutureAdapter<>();
@@ -2014,7 +2026,11 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     @Override public void onStateRestored(AffinityTopologyVersion topVer) throws IgniteCheckedException {
         long time = System.currentTimeMillis();
 
-        new IgniteThread(cctx.igniteInstanceName(), "db-checkpoint-thread", checkpointer).start();
+        IgniteThread cpThread = new IgniteThread(cctx.igniteInstanceName(), "db-checkpoint-thread", checkpointer);
+
+        cpThread.start();
+
+        checkpointerThread = cpThread;
 
         CheckpointProgressSnapshot chp = checkpointer.wakeupForCheckpoint(0, "node started");
 
@@ -4648,7 +4664,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             pageStore.ensure(grp.groupId(), p);
 
             if (pageStore.pages(grp.groupId(), p) <= 1) {
-                log.info("Partition [id=" + p + ", state=N/A (only file header) ]");
+                log.info("Partition [grp=" + grp.cacheOrGroupName() + ", id=" + p + ", state=N/A (only file header) ]");
 
                 continue;
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fe8c8cc5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheStartStopWithFreqCheckpointTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheStartStopWithFreqCheckpointTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheStartStopWithFreqCheckpointTest.java
new file mode 100644
index 0000000..8e00d88
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheStartStopWithFreqCheckpointTest.java
@@ -0,0 +1,177 @@
+/*
+ * 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.internal.processors.cache.persistence;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheRebalanceMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.WALMode;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Assert;
+
+/**
+ *
+ */
+public class IgnitePdsCacheStartStopWithFreqCheckpointTest extends GridCommonAbstractTest {
+    /** Caches. */
+    private static final int CACHES = 10;
+
+    /** Cache name. */
+    private static final String CACHE_NAME = "test";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setConsistentId(igniteInstanceName);
+
+        DataStorageConfiguration dsCfg = new DataStorageConfiguration()
+            .setWalMode(WALMode.LOG_ONLY)
+            .setCheckpointFrequency(1000)
+            .setDefaultDataRegionConfiguration(
+                new DataRegionConfiguration()
+                    .setMaxSize(512 * 1024 * 1024)
+                    .setPersistenceEnabled(true)
+            );
+
+        cfg.setDataStorageConfiguration(dsCfg);
+
+        CacheConfiguration[] ccfgs = new CacheConfiguration[CACHES];
+
+        for (int i = 0; i < ccfgs.length; i++)
+            ccfgs[i] = cacheConfiguration(i);
+
+        cfg.setCacheConfiguration(ccfgs);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    private CacheConfiguration cacheConfiguration(int cacheIdx) {
+        return new CacheConfiguration(CACHE_NAME + cacheIdx)
+            .setCacheMode(CacheMode.REPLICATED)
+            .setBackups(0)
+            .setRebalanceMode(CacheRebalanceMode.NONE);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /**
+     * Test checkpoint deadlock during caches start/stop process and frequent checkpoints is set.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCheckpointDeadlock() throws Exception {
+        IgniteEx crd = startGrid(0);
+
+        crd.cluster().active(true);
+
+        for (int cacheId = 0; cacheId < CACHES; cacheId++) {
+            IgniteCache<Object, Object> cache = crd.getOrCreateCache(CACHE_NAME + cacheId);
+
+            for (int key = 0; key < 4096; key++)
+                cache.put(key, key);
+        }
+
+        forceCheckpoint();
+
+        final AtomicBoolean stopFlag = new AtomicBoolean();
+
+        IgniteInternalFuture<?> cacheStartStopFut = GridTestUtils.runAsync(() -> {
+            while (!stopFlag.get()) {
+                List<String> cacheNames = new ArrayList<>();
+                for (int i = 0; i < CACHES / 2; i++)
+                    cacheNames.add(CACHE_NAME + i);
+
+                try {
+                    // Stop cache without destroy.
+                    crd.context().cache().dynamicDestroyCaches(cacheNames, false, false, false).get();
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteException("Failed to destroy cache", e);
+                }
+
+                List<CacheConfiguration> cachesToStart = new ArrayList<>();
+                for (int i = 0; i < CACHES / 2; i++)
+                    cachesToStart.add(cacheConfiguration(i));
+
+                crd.getOrCreateCaches(cachesToStart);
+            }
+        });
+
+        U.sleep(60_000);
+
+        log.info("Stopping caches start/stop process.");
+
+        stopFlag.set(true);
+
+        try {
+            cacheStartStopFut.get(30, TimeUnit.SECONDS);
+        }
+        catch (IgniteFutureTimeoutCheckedException e) {
+            U.dumpThreads(log);
+
+            log.warning("Caches start/stop future hangs. Interrupting checkpointer...");
+
+            interruptCheckpointer(crd);
+
+            // Should succeed.
+            cacheStartStopFut.get();
+
+            Assert.assertTrue("Checkpoint and exchange is probably in deadlock (see thread dump above for details).", false);
+        }
+    }
+
+    /**
+     * Interrupts checkpoint thread for given node.
+     *
+     * @param node Node.
+     */
+    private void interruptCheckpointer(IgniteEx node) {
+        GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager) node.context().cache().context().database();
+
+        dbMgr.checkpointerThread().interrupt();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fe8c8cc5/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
index c12f515..baaf882 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
@@ -19,6 +19,7 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.cache.persistence.IgniteDataStorageMetricsSelfTest;
+import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsCacheStartStopWithFreqCheckpointTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsCorruptedStoreTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsExchangeDuringCheckpointTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsPageSizesTest;
@@ -117,6 +118,8 @@ public class IgnitePdsTestSuite2 extends TestSuite {
         suite.addTestSuite(IgniteWalFlushFsyncWithDedicatedWorkerSelfTest.class);
 
         suite.addTestSuite(IgniteWalFlushFsyncWithMmapBufferSelfTest.class);
+
+        suite.addTestSuite(IgnitePdsCacheStartStopWithFreqCheckpointTest.class);
     }
 
     /**


[46/50] [abbrv] ignite git commit: IGNITE-10433 Web Console: Fixed "Import models" dialog cleanup on exit logic.

Posted by ag...@apache.org.
IGNITE-10433 Web Console: Fixed "Import models" dialog cleanup on exit logic.


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

Branch: refs/heads/ignite-9720
Commit: 3606b21bfe0d254864b284571a3f2692d4d4870a
Parents: 8e170d6
Author: Alexander Kalinin <ve...@yandex.ru>
Authored: Wed Nov 28 14:52:14 2018 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Wed Nov 28 14:52:14 2018 +0700

----------------------------------------------------------------------
 .../components/modal-import-models/service.js   | 24 +++++++++++++++-----
 1 file changed, 18 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3606b21b/modules/web-console/frontend/app/components/page-configure/components/modal-import-models/service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/page-configure/components/modal-import-models/service.js b/modules/web-console/frontend/app/components/page-configure/components/modal-import-models/service.js
index 3c0ecbb..e17721db 100644
--- a/modules/web-console/frontend/app/components/page-configure/components/modal-import-models/service.js
+++ b/modules/web-console/frontend/app/components/page-configure/components/modal-import-models/service.js
@@ -16,12 +16,18 @@
  */
 
 export default class ModalImportModels {
-    static $inject = ['$modal', 'AgentManager', '$uiRouter'];
-    constructor($modal, AgentManager, $uiRouter) {
-        Object.assign(this, {$modal, AgentManager, $uiRouter});
+    static $inject = ['$modal', '$uiRouter', 'AgentManager'];
+
+    constructor($modal, $uiRouter, AgentManager) {
+        this.$modal = $modal;
+        this.$uiRouter = $uiRouter;
+        this.AgentManager = AgentManager;
     }
+
     _goToDynamicState() {
-        if (this._state) this.$uiRouter.stateRegistry.deregister(this._state);
+        if (this._state)
+            this.$uiRouter.stateRegistry.deregister(this._state);
+
         this._state = this.$uiRouter.stateRegistry.register({
             name: 'importModels',
             parent: this.$uiRouter.stateService.current,
@@ -29,11 +35,15 @@ export default class ModalImportModels {
                 this._open();
             },
             onExit: () => {
+                this.AgentManager.stopWatch();
+
                 this._modal && this._modal.hide();
             }
         });
+
         return this.$uiRouter.stateService.go(this._state, this.$uiRouter.stateService.params);
     }
+
     _open() {
         this._modal = this.$modal({
             template: `
@@ -46,10 +56,12 @@ export default class ModalImportModels {
             controllerAs: '$ctrl',
             show: false
         });
+
         return this.AgentManager.startAgentWatch('Back', this.$uiRouter.globals.current.name)
-        .then(() => this._modal.$promise)
-        .then(() => this._modal.show());
+            .then(() => this._modal.$promise)
+            .then(() => this._modal.show());
     }
+
     open() {
         this._goToDynamicState();
     }


[20/50] [abbrv] ignite git commit: IGNITE-9937 Primary response error can be lost due to unwrapping a key - Fixes #5078.

Posted by ag...@apache.org.
IGNITE-9937 Primary response error can be lost due to unwrapping a key - Fixes #5078.

Signed-off-by: Pavel Kovalenko <jo...@gmail.com>


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

Branch: refs/heads/ignite-9720
Commit: 7e1d17830429a78ca62e2f007fece7de6466eb0f
Parents: e8eeea3
Author: Roman Guseinov <gr...@gmail.com>
Authored: Mon Nov 26 16:57:47 2018 +0300
Committer: Pavel Kovalenko <jo...@gmail.com>
Committed: Mon Nov 26 16:57:47 2018 +0300

----------------------------------------------------------------------
 .../dht/atomic/GridDhtAtomicCache.java          |  30 ++++-
 .../GridNearAtomicAbstractUpdateFuture.java     |  43 ++++++-
 .../cache/store/CacheStoreWriteErrorTest.java   | 127 +++++++++++++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 4 files changed, 196 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7e1d1783/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 86d7b3c..74be8e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -34,6 +34,7 @@ import javax.cache.processor.EntryProcessorResult;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.binary.BinaryInvalidTypeException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.NodeStoppingException;
@@ -2725,6 +2726,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
             final GridDhtAtomicAbstractUpdateFuture dhtFut = dhtUpdRes.dhtFuture();
 
+            Collection<Object> failedToUnwrapKeys = null;
+
             // Avoid iterator creation.
             for (int i = 0; i < entries.size(); i++) {
                 GridDhtCacheEntry entry = entries.get(i);
@@ -2737,9 +2740,26 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                     continue;
                 }
 
-                if (storeErr != null &&
-                    storeErr.failedKeys().contains(entry.key().value(ctx.cacheObjectContext(), false)))
-                    continue;
+                if (storeErr != null) {
+                    Object key = entry.key();
+
+                    try {
+                        key = entry.key().value(ctx.cacheObjectContext(), false);
+                    }
+                    catch (BinaryInvalidTypeException e) {
+                        if (log.isDebugEnabled()) {
+                            if (failedToUnwrapKeys == null)
+                                failedToUnwrapKeys = new ArrayList<>();
+
+                            // To limit keys count in log message.
+                            if (failedToUnwrapKeys.size() < 5)
+                                failedToUnwrapKeys.add(key);
+                        }
+                    }
+
+                    if (storeErr.failedKeys().contains(key))
+                        continue;
+                }
 
                 try {
                     // We are holding java-level locks on entries at this point.
@@ -2868,6 +2888,10 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                 dhtUpdRes.processedEntriesCount(firstEntryIdx + i + 1);
             }
 
+            if (failedToUnwrapKeys != null) {
+                log.warning("Failed to get values of keys: " + failedToUnwrapKeys +
+                    " (the binary objects will be used instead).");
+            }
         }
         catch (IgniteCheckedException e) {
             res.addFailedKeys(putMap != null ? putMap.keySet() : rmvKeys, e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7e1d1783/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
index f91e3f3..983b094 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
@@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import javax.cache.expiry.ExpiryPolicy;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.binary.BinaryInvalidTypeException;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteInternalFuture;
@@ -399,10 +400,46 @@ public abstract class GridNearAtomicAbstractUpdateFuture extends GridCacheFuture
 
         Collection<Object> keys = new ArrayList<>(keys0.size());
 
-        for (KeyCacheObject key : keys0)
-            keys.add(cctx.cacheObjectContext().unwrapBinaryIfNeeded(key, keepBinary, false));
+        Collection<Object> failedToUnwrapKeys = null;
 
-        err.add(keys, res.error(), req.topologyVersion());
+        Exception suppressedErr = null;
+
+        for (KeyCacheObject key : keys0) {
+            try {
+                keys.add(cctx.cacheObjectContext().unwrapBinaryIfNeeded(key, keepBinary, false));
+            }
+            catch (BinaryInvalidTypeException e) {
+                keys.add(cctx.toCacheKeyObject(key));
+
+                if (log.isDebugEnabled()) {
+                    if (failedToUnwrapKeys == null)
+                        failedToUnwrapKeys = new ArrayList<>();
+
+                    // To limit keys count in log message.
+                    if (failedToUnwrapKeys.size() < 5)
+                        failedToUnwrapKeys.add(key);
+                }
+
+                suppressedErr = e;
+            }
+            catch (Exception e) {
+                keys.add(cctx.toCacheKeyObject(key));
+
+                suppressedErr = e;
+            }
+        }
+
+        if (failedToUnwrapKeys != null) {
+            log.warning("Failed to unwrap keys: " + failedToUnwrapKeys +
+                " (the binary objects will be used instead).");
+        }
+
+        IgniteCheckedException error = res.error();
+
+        if (suppressedErr != null)
+            error.addSuppressed(suppressedErr);
+
+        err.add(keys, error, req.topologyVersion());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7e1d1783/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreWriteErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreWriteErrorTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreWriteErrorTest.java
new file mode 100644
index 0000000..fce1f5d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreWriteErrorTest.java
@@ -0,0 +1,127 @@
+/*
+ * 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.cache.store;
+
+import com.google.common.base.Throwables;
+import java.util.HashMap;
+import java.util.concurrent.Callable;
+import javax.cache.Cache;
+import javax.cache.configuration.FactoryBuilder;
+import javax.cache.integration.CacheLoaderException;
+import javax.cache.integration.CacheWriterException;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * This class tests handling exceptions from {@link CacheStore#write(Cache.Entry)}.
+ */
+public class CacheStoreWriteErrorTest extends GridCommonAbstractTest {
+    /** */
+    public static final String CACHE_NAME = "cache";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        CacheConfiguration cacheCfg = new CacheConfiguration(CACHE_NAME)
+            .setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC)
+            .setCacheStoreFactory(FactoryBuilder.factoryOf(ThrowableCacheStore.class))
+            .setWriteThrough(true)
+            .setStoreKeepBinary(true);
+
+        return super.getConfiguration(gridName)
+            .setCacheConfiguration(cacheCfg);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * Checks primary error while saving batch with one entry.
+     */
+    public void testPrimaryErrorForBatchSize1() {
+        checkPrimaryError(1);
+    }
+
+    /**
+     * Checks primary error while saving batch with two entries.
+     */
+    public void testPrimaryErrorForBatchSize2() {
+        checkPrimaryError(2);
+    }
+
+    /**
+     * Checks that primary error ({@link CacheWriterException}) is not lost due to unwrapping a key.
+     *
+     * @param batchSize Batch size.
+     */
+    private void checkPrimaryError(int batchSize) {
+        Throwable t = GridTestUtils.assertThrows(log,
+            new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    try (Ignite grid = startGrid()) {
+                        IgniteCache<BinaryObject, String> cache = grid.cache(CACHE_NAME);
+
+                        HashMap<BinaryObject, String> batch = new HashMap<>();
+
+                        for (int i = 0; i < batchSize; i++) {
+                            BinaryObject key = grid.binary().builder("KEY_TYPE_NAME").setField("id", i).build();
+
+                            batch.put(key, "VALUE");
+                        }
+
+                        cache.putAllAsync(batch).get();
+                    }
+
+                    return null;
+                }
+            }, CacheWriterException.class, null);
+
+        assertTrue("Stacktrace should contain the message of the original exception",
+            Throwables.getStackTraceAsString(t).contains(ThrowableCacheStore.EXCEPTION_MESSAGE));
+    }
+
+    /**
+     * {@link CacheStore} implementation which throws {@link RuntimeException} for every write operation.
+     */
+    public static class ThrowableCacheStore extends CacheStoreAdapter<Object, Object> {
+        /** */
+        private static final String EXCEPTION_MESSAGE = "WRITE CACHE STORE EXCEPTION";
+
+        /** {@inheritDoc} */
+        @Override public Object load(Object o) throws CacheLoaderException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<?, ?> entry) throws CacheWriterException {
+            throw new RuntimeException(EXCEPTION_MESSAGE);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object o) throws CacheWriterException {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7e1d1783/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 5d7b306..52e2ba2 100755
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -22,6 +22,7 @@ import junit.framework.TestSuite;
 import org.apache.ignite.cache.IgniteCacheEntryProcessorSequentialCallTest;
 import org.apache.ignite.cache.IgniteWarmupClosureSelfTest;
 import org.apache.ignite.cache.store.CacheStoreReadFromBackupTest;
+import org.apache.ignite.cache.store.CacheStoreWriteErrorTest;
 import org.apache.ignite.cache.store.CacheTransactionalStoreReadFromBackupTest;
 import org.apache.ignite.cache.store.GridCacheBalancingStoreSelfTest;
 import org.apache.ignite.cache.store.GridCacheLoadOnlyStoreAdapterSelfTest;
@@ -333,6 +334,7 @@ public class IgniteCacheTestSuite extends TestSuite {
 
         suite.addTestSuite(GridStoreLoadCacheTest.class);
         suite.addTestSuite(CacheStoreReadFromBackupTest.class);
+        suite.addTestSuite(CacheStoreWriteErrorTest.class);
         suite.addTestSuite(CacheTransactionalStoreReadFromBackupTest.class);
 
         //suite.addTestSuite(CacheAtomicSingleMessageCountSelfTest.class);


[39/50] [abbrv] ignite git commit: IGNITE-1793 [Failed Test] IgnitePartitionedCountDownLatchSelfTest.testLatch hangs on TC sometimes - Fixes #2882.

Posted by ag...@apache.org.
IGNITE-1793
[Failed Test] IgnitePartitionedCountDownLatchSelfTest.testLatch hangs on TC sometimes - Fixes #2882.

Signed-off-by: Dmitriy Pavlov <dp...@apache.org>


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

Branch: refs/heads/ignite-9720
Commit: 07f1dbfcb899d604d6637a71c956b14e6d52ac9c
Parents: 9d3c757
Author: vd-pyatkov <vp...@gridgain.com>
Authored: Tue Nov 27 19:58:54 2018 +0300
Committer: Dmitriy Pavlov <dp...@apache.org>
Committed: Tue Nov 27 19:58:54 2018 +0300

----------------------------------------------------------------------
 .../partitioned/IgnitePartitionedCountDownLatchSelfTest.java    | 5 -----
 1 file changed, 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/07f1dbfc/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/IgnitePartitionedCountDownLatchSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/IgnitePartitionedCountDownLatchSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/IgnitePartitionedCountDownLatchSelfTest.java
index fc9356e..fbd3779 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/IgnitePartitionedCountDownLatchSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/IgnitePartitionedCountDownLatchSelfTest.java
@@ -30,9 +30,4 @@ public class IgnitePartitionedCountDownLatchSelfTest extends IgniteCountDownLatc
     @Override protected CacheMode atomicsCacheMode() {
         return PARTITIONED;
     }
-
-    /** {@inheritDoc} */
-    @Override public void testLatch() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1793");
-    }
 }


[18/50] [abbrv] ignite git commit: IGNITE-10330: Disk page compression. - Fixes #5200.

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/compress/licenses/jnr-posix-LICENSE.txt
----------------------------------------------------------------------
diff --git a/modules/compress/licenses/jnr-posix-LICENSE.txt b/modules/compress/licenses/jnr-posix-LICENSE.txt
new file mode 100644
index 0000000..58011d4
--- /dev/null
+++ b/modules/compress/licenses/jnr-posix-LICENSE.txt
@@ -0,0 +1,2486 @@
+jnr-posix is released under a tri EPL/GPL/LGPL license. You can use it,
+redistribute it and/or modify it under the terms of the:
+
+  Eclipse Public License version 1.0
+  GNU General Public License version 2
+  GNU Lesser General Public License version 2.1
+
+The complete text of the Eclipse Public License is as follows:
+
+  Eclipse Public License - v 1.0
+
+  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 Losses 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.
+
+The complete text of the Eclipse Public License is as follows:
+
+  Eclipse Public License - v 1.0
+
+  THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS COMMON
+  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 Losses
+  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 a Contributor with
+  respect to a patent applicable to software (including a cross-claim or
+  counterclaim in a lawsuit), then any patent licenses granted by that
+  Contributor to such Recipient under this Agreement shall terminate as
+  of the date such litigation is filed. In addition, 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. IBM is the initial
+  Agreement Steward. IBM 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.
+
+The complete text of the GNU General Public License v2 is as follows:
+
+          GNU GENERAL PUBLIC LICENSE
+             Version 2, June 1991
+
+   Copyright (C) 1989, 1991 Free Software Foundation, Inc.
+                         59 Temple Place, Suite 330, Boston, MA  02111-1307  USA
+   Everyone is permitted to copy and distribute verbatim copies
+   of this license document, but changing it is not allowed.
+
+            Preamble
+
+    The licenses for most software are designed to take away your
+  freedom to share and change it.  By contrast, the GNU General Public
+  License is intended to guarantee your freedom to share and change free
+  software--to make sure the software is free for all its users.  This
+  General Public License applies to most of the Free Software
+  Foundation's software and to any other program whose authors commit to
+  using it.  (Some other Free Software Foundation software is covered by
+  the GNU Library General Public License instead.)  You can apply it to
+  your programs, too.
+
+    When we speak of free software, we are referring to freedom, not
+  price.  Our General Public Licenses are designed to make sure that you
+  have the freedom to distribute copies of free software (and charge for
+  this service if you wish), that you receive source code or can get it
+  if you want it, that you can change the software or use pieces of it
+  in new free programs; and that you know you can do these things.
+
+    To protect your rights, we need to make restrictions that forbid
+  anyone to deny you these rights or to ask you to surrender the rights.
+  These restrictions translate to certain responsibilities for you if you
+  distribute copies of the software, or if you modify it.
+
+    For example, if you distribute copies of such a program, whether
+  gratis or for a fee, you must give the recipients all the rights that
+  you have.  You must make sure that they, too, receive or can get the
+  source code.  And you must show them these terms so they know their
+  rights.
+
+    We protect your rights with two steps: (1) copyright the software, and
+  (2) offer you this license which gives you legal permission to copy,
+  distribute and/or modify the software.
+
+    Also, for each author's protection and ours, we want to make certain
+  that everyone understands that there is no warranty for this free
+  software.  If the software is modified by someone else and passed on, we
+  want its recipients to know that what they have is not the original, so
+  that any problems introduced by others will not reflect on the original
+  authors' reputations.
+
+    Finally, any free program is threatened constantly by software
+  patents.  We wish to avoid the danger that redistributors of a free
+  program will individually obtain patent licenses, in effect making the
+  program proprietary.  To prevent this, we have made it clear that any
+  patent must be licensed for everyone's free use or not licensed at all.
+
+    The precise terms and conditions for copying, distribution and
+  modification follow.
+
+          GNU GENERAL PUBLIC LICENSE
+     TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION
+
+    0. This License applies to any program or other work which contains
+  a notice placed by the copyright holder saying it may be distributed
+  under the terms of this General Public License.  The "Program", below,
+  refers to any such program or work, and a "work based on the Program"
+  means either the Program or any derivative work under copyright law:
+  that is to say, a work containing the Program or a portion of it,
+  either verbatim or with modifications and/or translated into another
+  language.  (Hereinafter, translation is included without limitation in
+  the term "modification".)  Each licensee is addressed as "you".
+
+  Activities other than copying, distribution and modification are not
+  covered by this License; they are outside its scope.  The act of
+  running the Program is not restricted, and the output from the Program
+  is covered only if its contents constitute a work based on the
+  Program (independent of having been made by running the Program).
+  Whether that is true depends on what the Program does.
+
+    1. You may copy and distribute verbatim copies of the Program's
+  source code as you receive it, in any medium, provided that you
+  conspicuously and appropriately publish on each copy an appropriate
+  copyright notice and disclaimer of warranty; keep intact all the
+  notices that refer to this License and to the absence of any warranty;
+  and give any other recipients of the Program a copy of this License
+  along with the Program.
+
+  You may charge a fee for the physical act of transferring a copy, and
+  you may at your option offer warranty protection in exchange for a fee.
+
+    2. You may modify your copy or copies of the Program or any portion
+  of it, thus forming a work based on the Program, and copy and
+  distribute such modifications or work under the terms of Section 1
+  above, provided that you also meet all of these conditions:
+
+      a) You must cause the modified files to carry prominent notices
+      stating that you changed the files and the date of any change.
+
+      b) You must cause any work that you distribute or publish, that in
+      whole or in part contains or is derived from the Program or any
+      part thereof, to be licensed as a whole at no charge to all third
+      parties under the terms of this License.
+
+      c) If the modified program normally reads commands interactively
+      when run, you must cause it, when started running for such
+      interactive use in the most ordinary way, to print or display an
+      announcement including an appropriate copyright notice and a
+      notice that there is no warranty (or else, saying that you provide
+      a warranty) and that users may redistribute the program under
+      these conditions, and telling the user how to view a copy of this
+      License.  (Exception: if the Program itself is interactive but
+      does not normally print such an announcement, your work based on
+      the Program is not required to print an announcement.)
+
+  These requirements apply to the modified work as a whole.  If
+  identifiable sections of that work are not derived from the Program,
+  and can be reasonably considered independent and separate works in
+  themselves, then this License, and its terms, do not apply to those
+  sections when you distribute them as separate works.  But when you
+  distribute the same sections as part of a whole which is a work based
+  on the Program, the distribution of the whole must be on the terms of
+  this License, whose permissions for other licensees extend to the
+  entire whole, and thus to each and every part regardless of who wrote it.
+
+  Thus, it is not the intent of this section to claim rights or contest
+  your rights to work written entirely by you; rather, the intent is to
+  exercise the right to control the distribution of derivative or
+  collective works based on the Program.
+
+  In addition, mere aggregation of another work not based on the Program
+  with the Program (or with a work based on the Program) on a volume of
+  a storage or distribution medium does not bring the other work under
+  the scope of this License.
+
+    3. You may copy and distribute the Program (or a work based on it,
+  under Section 2) in object code or executable form under the terms of
+  Sections 1 and 2 above provided that you also do one of the following:
+
+      a) Accompany it with the complete corresponding machine-readable
+      source code, which must be distributed under the terms of Sections
+      1 and 2 above on a medium customarily used for software interchange; or,
+
+      b) Accompany it with a written offer, valid for at least three
+      years, to give any third party, for a charge no more than your
+      cost of physically performing source distribution, a complete
+      machine-readable copy of the corresponding source code, to be
+      distributed under the terms of Sections 1 and 2 above on a medium
+      customarily used for software interchange; or,
+
+      c) Accompany it with the information you received as to the offer
+      to distribute corresponding source code.  (This alternative is
+      allowed only for noncommercial distribution and only if you
+      received the program in object code or executable form with such
+      an offer, in accord with Subsection b above.)
+
+  The source code for a work means the preferred form of the work for
+  making modifications to it.  For an executable work, complete source
+  code means all the source code for all modules it contains, plus any
+  associated interface definition files, plus the scripts used to
+  control compilation and installation of the executable.  However, as a
+  special exception, the source code distributed need not include
+  anything that is normally distributed (in either source or binary
+  form) with the major components (compiler, kernel, and so on) of the
+  operating system on which the executable runs, unless that component
+  itself accompanies the executable.
+
+  If distribution of executable or object code is made by offering
+  access to copy from a designated place, then offering equivalent
+  access to copy the source code from the same place counts as
+  distribution of the source code, even though third parties are not
+  compelled to copy the source along with the object code.
+
+    4. You may not copy, modify, sublicense, or distribute the Program
+  except as expressly provided under this License.  Any attempt
+  otherwise to copy, modify, sublicense or distribute the Program is
+  void, and will automatically terminate your rights under this License.
+  However, parties who have received copies, or rights, from you under
+  this License will not have their licenses terminated so long as such
+  parties remain in full compliance.
+
+    5. You are not required to accept this License, since you have not
+  signed it.  However, nothing else grants you permission to modify or
+  distribute the Program or its derivative works.  These actions are
+  prohibited by law if you do not accept this License.  Therefore, by
+  modifying or distributing the Program (or any work based on the
+  Program), you indicate your acceptance of this License to do so, and
+  all its terms and conditions for copying, distributing or modifying
+  the Program or works based on it.
+
+    6. Each time you redistribute the Program (or any work based on the
+  Program), the recipient automatically receives a license from the
+  original licensor to copy, distribute or modify the Program subject to
+  these terms and conditions.  You may not impose any further
+  restrictions on the recipients' exercise of the rights granted herein.
+  You are not responsible for enforcing compliance by third parties to
+  this License.
+
+    7. If, as a consequence of a court judgment or allegation of patent
+  infringement or for any other reason (not limited to patent issues),
+  conditions are imposed on you (whether by court order, agreement or
+  otherwise) that contradict the conditions of this License, they do not
+  excuse you from the conditions of this License.  If you cannot
+  distribute so as to satisfy simultaneously your obligations under this
+  License and any other pertinent obligations, then as a consequence you
+  may not distribute the Program at all.  For example, if a patent
+  license would not permit royalty-free redistribution of the Program by
+  all those who receive copies directly or indirectly through you, then
+  the only way you could satisfy both it and this License would be to
+  refrain entirely from distribution of the Program.
+
+  If any portion of this section is held invalid or unenforceable under
+  any particular circumstance, the balance of the section is intended to
+  apply and the section as a whole is intended to apply in other
+  circumstances.
+
+  It is not the purpose of this section to induce you to infringe any
+  patents or other property right claims or to contest validity of any
+  such claims; this section has the sole purpose of protecting the
+  integrity of the free software distribution system, which is
+  implemented by public license practices.  Many people have made
+  generous contributions to the wide range of software distributed
+  through that system in reliance on consistent application of that
+  system; it is up to the author/donor to decide if he or she is willing
+  to distribute software through any other system and a licensee cannot
+  impose that choice.
+
+  This section is intended to make thoroughly clear what is believed to
+  be a consequence of the rest of this License.
+
+    8. If the distribution and/or use of the Program is restricted in
+  certain countries either by patents or by copyrighted interfaces, the
+  original copyright holder who places the Program under this License
+  may add an explicit geographical distribution limitation excluding
+  those countries, so that distribution is permitted only in or among
+  countries not thus excluded.  In such case, this License incorporates
+  the limitation as if written in the body of this License.
+
+    9. The Free Software Foundation may publish revised and/or new versions
+  of the General Public License from time to time.  Such new versions will
+  be similar in spirit to the present version, but may differ in detail to
+  address new problems or concerns.
+
+  Each version is given a distinguishing version number.  If the Program
+  specifies a version number of this License which applies to it and "any
+  later version", you have the option of following the terms and conditions
+  either of that version or of any later version published by the Free
+  Software Foundation.  If the Program does not specify a version number of
+  this License, you may choose any version ever published by the Free Software
+  Foundation.
+
+    10. If you wish to incorporate parts of the Program into other free
+  programs whose distribution conditions are different, write to the author
+  to ask for permission.  For software which is copyrighted by the Free
+  Software Foundation, write to the Free Software Foundation; we sometimes
+  make exceptions for this.  Our decision will be guided by the two goals
+  of preserving the free status of all derivatives of our free software and
+  of promoting the sharing and reuse of software generally.
+
+            NO WARRANTY
+
+    11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY
+  FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW.  EXCEPT WHEN
+  OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES
+  PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED
+  OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF
+  MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE.  THE ENTIRE RISK AS
+  TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH YOU.  SHOULD THE
+  PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL NECESSARY SERVICING,
+  REPAIR OR CORRECTION.
+
+    12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING
+  WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR
+  REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES,
+  INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING
+  OUT OF THE USE OR INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED
+  TO LOSS OF DATA OR DATA BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY
+  YOU OR THIRD PARTIES OR A FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER
+  PROGRAMS), EVEN IF SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE
+  POSSIBILITY OF SUCH DAMAGES.
+
+           END OF TERMS AND CONDITIONS
+
+The complete text of the GNU Lesser General Public License 2.1 is as follows:
+
+        GNU LESSER GENERAL PUBLIC LICENSE
+             Version 2.1, February 1999
+
+   Copyright (C) 1991, 1999 Free Software Foundation, Inc.
+       59 Temple Place, Suite 330, Boston, MA  02111-1307  USA
+   Everyone is permitted to copy and distribute verbatim copies
+   of this license document, but changing it is not allowed.
+
+  [This is the first released version of the Lesser GPL.  It also counts
+   as the successor of the GNU Library Public License, version 2, hence
+   the version number 2.1.]
+
+            Preamble
+
+    The licenses for most software are designed to take away your
+  freedom to share and change it.  By contrast, the GNU General Public
+  Licenses are intended to guarantee your freedom to share and change
+  free software--to make sure the software is free for all its users.
+
+    This license, the Lesser General Public License, applies to some
+  specially designated software packages--typically libraries--of the
+  Free Software Foundation and other authors who decide to use it.  You
+  can use it too, but we suggest you first think carefully about whether
+  this license or the ordinary General Public License is the better
+  strategy to use in any particular case, based on the explanations below.
+
+    When we speak of free software, we are referring to freedom of use,
+  not price.  Our General Public Licenses are designed to make sure that
+  you have the freedom to distribute copies of free software (and charge
+  for this service if you wish); that you receive source code or can get
+  it if you want it; that you can change the software and use pieces of
+  it in new free programs; and that you are informed that you can do
+  these things.
+
+    To protect your rights, we need to make restrictions that forbid
+  distributors to deny you these rights or to ask you to surrender these
+  rights.  These restrictions translate to certain responsibilities for
+  you if you distribute copies of the library or if you modify it.
+
+    For example, if you distribute copies of the library, whether gratis
+  or for a fee, you must give the recipients all the rights that we gave
+  you.  You must make sure that they, too, receive or can get the source
+  code.  If you link other code with the library, you must provide
+  complete object files to the recipients, so that they can relink them
+  with the library after making changes to the library and recompiling
+  it.  And you must show them these terms so they know their rights.
+
+    We protect your rights with a two-step method: (1) we copyright the
+  library, and (2) we offer you this license, which gives you legal
+  permission to copy, distribute and/or modify the library.
+
+    To protect each distributor, we want to make it very clear that
+  there is no warranty for the free library.  Also, if the library is
+  modified by someone else and passed on, the recipients should know
+  that what they have is not the original version, so that the original
+  author's reputation will not be affected by problems that might be
+  introduced by others.
+
+    Finally, software patents pose a constant threat to the existence of
+  any free program.  We wish to make sure that a company cannot
+  effectively restrict the users of a free program by obtaining a
+  restrictive license from a patent holder.  Therefore, we insist that
+  any patent license obtained for a version of the library must be
+  consistent with the full freedom of use specified in this license.
+
+    Most GNU software, including some libraries, is covered by the
+  ordinary GNU General Public License.  This license, the GNU Lesser
+  General Public License, applies to certain designated libraries, and
+  is quite different from the ordinary General Public License.  We use
+  this license for certain libraries in order to permit linking those
+  libraries into non-free programs.
+
+    When a program is linked with a library, whether statically or using
+  a shared library, the combination of the two is legally speaking a
+  combined work, a derivative of the original library.  The ordinary
+  General Public License therefore permits such linking only if the
+  entire combination fits its criteria of freedom.  The Lesser General
+  Public License permits more lax criteria for linking other code with
+  the library.
+
+    We call this license the "Lesser" General Public License because it
+  does Less to protect the user's freedom than the ordinary General
+  Public License.  It also provides other free software developers Less
+  of an advantage over competing non-free programs.  These disadvantages
+  are the reason we use the ordinary General Public License for many
+  libraries.  However, the Lesser license provides advantages in certain
+  special circumstances.
+
+    For example, on rare occasions, there may be a special need to
+  encourage the widest possible use of a certain library, so that it becomes
+  a de-facto standard.  To achieve this, non-free programs must be
+  allowed to use the library.  A more frequent case is that a free
+  library does the same job as widely used non-free libraries.  In this
+  case, there is little to gain by limiting the free library to free
+  software only, so we use the Lesser General Public License.
+
+    In other cases, permission to use a particular library in non-free
+  programs enables a greater number of people to use a large body of
+  free software.  For example, permission to use the GNU C Library in
+  non-free programs enables many more people to use the whole GNU
+  operating system, as well as its variant, the GNU/Linux operating
+  system.
+
+    Although the Lesser General Public License is Less protective of the
+  users' freedom, it does ensure that the user of a program that is
+  linked with the Library has the freedom and the wherewithal to run
+  that program using a modified version of the Library.
+
+    The precise terms and conditions for copying, distribution and
+  modification follow.  Pay close attention to the difference between a
+  "work based on the library" and a "work that uses the library".  The
+  former contains code derived from the library, whereas the latter must
+  be combined with the library in order to run.
+
+        GNU LESSER GENERAL PUBLIC LICENSE
+     TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION
+
+    0. This License Agreement applies to any software library or other
+  program which contains a notice placed by the copyright holder or
+  other authorized party saying it may be distributed under the terms of
+  this Lesser General Public License (also called "this License").
+  Each licensee is addressed as "you".
+
+    A "library" means a collection of software functions and/or data
+  prepared so as to be conveniently linked with application programs
+  (which use some of those functions and data) to form executables.
+
+    The "Library", below, refers to any such software library or work
+  which has been distributed under these terms.  A "work based on the
+  Library" means either the Library or any derivative work under
+  copyright law: that is to say, a work containing the Library or a
+  portion of it, either verbatim or with modifications and/or translated
+  straightforwardly into another language.  (Hereinafter, translation is
+  included without limitation in the term "modification".)
+
+    "Source code" for a work means the preferred form of the work for
+  making modifications to it.  For a library, complete source code means
+  all the source code for all modules it contains, plus any associated
+  interface definition files, plus the scripts used to control compilation
+  and installation of the library.
+
+    Activities other than copying, distribution and modification are not
+  covered by this License; they are outside its scope.  The act of
+  running a program using the Library is not restricted, and output from
+  such a program is covered only if its contents constitute a work based
+  on the Library (independent of the use of the Library in a tool for
+  writing it).  Whether that is true depends on what the Library does
+  and what the program that uses the Library does.
+  
+    1. You may copy and distribute verbatim copies of the Library's
+  complete source code as you receive it, in any medium, provided that
+  you conspicuously and appropriately publish on each copy an
+  appropriate copyright notice and disclaimer of warranty; keep intact
+  all the notices that refer to this License and to the absence of any
+  warranty; and distribute a copy of this License along with the
+  Library.
+
+    You may charge a fee for the physical act of transferring a copy,
+  and you may at your option offer warranty protection in exchange for a
+  fee.
+
+    2. You may modify your copy or copies of the Library or any portion
+  of it, thus forming a work based on the Library, and copy and
+  distribute such modifications or work under the terms of Section 1
+  above, provided that you also meet all of these conditions:
+
+      a) The modified work must itself be a software library.
+
+      b) You must cause the files modified to carry prominent notices
+      stating that you changed the files and the date of any change.
+
+      c) You must cause the whole of the work to be licensed at no
+      charge to all third parties under the terms of this License.
+
+      d) If a facility in the modified Library refers to a function or a
+      table of data to be supplied by an application program that uses
+      the facility, other than as an argument passed when the facility
+      is invoked, then you must make a good faith effort to ensure that,
+      in the event an application does not supply such function or
+      table, the facility still operates, and performs whatever part of
+      its purpose remains meaningful.
+
+      (For example, a function in a library to compute square roots has
+      a purpose that is entirely well-defined independent of the
+      application.  Therefore, Subsection 2d requires that any
+      application-supplied function or table used by this function must
+      be optional: if the application does not supply it, the square
+      root function must still compute square roots.)
+
+  These requirements apply to the modified work as a whole.  If
+  identifiable sections of that work are not derived from the Library,
+  and can be reasonably considered independent and separate works in
+  themselves, then this License, and its terms, do not apply to those
+  sections when you distribute them as separate works.  But when you
+  distribute the same sections as part of a whole which is a work based
+  on the Library, the distribution of the whole must be on the terms of
+  this License, whose permissions for other licensees extend to the
+  entire whole, and thus to each and every part regardless of who wrote
+  it.
+
+  Thus, it is not the intent of this section to claim rights or contest
+  your rights to work written entirely by you; rather, the intent is to
+  exercise the right to control the distribution of derivative or
+  collective works based on the Library.
+
+  In addition, mere aggregation of another work not based on the Library
+  with the Library (or with a work based on the Library) on a volume of
+  a storage or distribution medium does not bring the other work under
+  the scope of this License.
+
+    3. You may opt to apply the terms of the ordinary GNU General Public
+  License instead of this License to a given copy of the Library.  To do
+  this, you must alter all the notices that refer to this License, so
+  that they refer to the ordinary GNU General Public License, version 2,
+  instead of to this License.  (If a newer version than version 2 of the
+  ordinary GNU General Public License has appeared, then you can specify
+  that version instead if you wish.)  Do not make any other change in
+  these notices.
+
+    Once this change is made in a given copy, it is irreversible for
+  that copy, so the ordinary GNU General Public License applies to all
+  subsequent copies and derivative works made from that copy.
+
+    This option is useful when you wish to copy part of the code of
+  the Library into a program that is not a library.
+
+    4. You may copy and distribute the Library (or a portion or
+  derivative of it, under Section 2) in object code or executable form
+  under the terms of Sections 1 and 2 above provided that you accompany
+  it with the complete corresponding machine-readable source code, which
+  must be distributed under the terms of Sections 1 and 2 above on a
+  medium customarily used for software interchange.
+
+    If distribution of object code is made by offering access to copy
+  from a designated place, then offering equivalent access to copy the
+  source code from the same place satisfies the requirement to
+  distribute the source code, even though third parties are not
+  compelled to copy the source along with the object code.
+
+    5. A program that contains no derivative of any portion of the
+  Library, but is designed to work with the Library by being compiled or
+  linked with it, is called a "work that uses the Library".  Such a
+  work, in isolation, is not a derivative work of the Library, and
+  therefore falls outside the scope of this License.
+
+    However, linking a "work that uses the Library" with the Library
+  creates an executable that is a derivative of the Library (because it
+  contains portions of the Library), rather than a "work that uses the
+  library".  The executable is therefore covered by this License.
+  Section 6 states terms for distribution of such executables.
+
+    When a "work that uses the Library" uses material from a header file
+  that is part of the Library, the object code for the work may be a
+  derivative work of the Library even though the source code is not.
+  Whether this is true is especially significant if the work can be
+  linked without the Library, or if the work is itself a library.  The
+  threshold for this to be true is not precisely defined by law.
+
+    If such an object file uses only numerical parameters, data
+  structure layouts and accessors, and small macros and small inline
+  functions (ten lines or less in length), then the use of the object
+  file is unrestricted, regardless of whether it is legally a derivative
+  work.  (Executables containing this object code plus portions of the
+  Library will still fall under Section 6.)
+
+    Otherwise, if the work is a derivative of the Library, you may
+  distribute the object code for the work under the terms of Section 6.
+  Any executables containing that work also fall under Section 6,
+  whether or not they are linked directly with the Library itself.
+
+    6. As an exception to the Sections above, you may also combine or
+  link a "work that uses the Library" with the Library to produce a
+  work containing portions of the Library, and distribute that work
+  under terms of your choice, provided that the terms permit
+  modification of the work for the customer's own use and reverse
+  engineering for debugging such modifications.
+
+    You must give prominent notice with each copy of the work that the
+  Library is used in it and that the Library and its use are covered by
+  this License.  You must supply a copy of this License.  If the work
+  during execution displays copyright notices, you must include the
+  copyright notice for the Library among them, as well as a reference
+  directing the user to the copy of this License.  Also, you must do one
+  of these things:
+
+      a) Accompany the work with the complete corresponding
+      machine-readable source code for the Library including whatever
+      changes were used in the work (which must be distributed under
+      Sections 1 and 2 above); and, if the work is an executable linked
+      with the Library, with the complete machine-readable "work that
+      uses the Library", as object code and/or source code, so that the
+      user can modify the Library and then relink to produce a modified
+      executable containing the modified Library.  (It is understood
+      that the user who changes the contents of definitions files in the
+      Library will not necessarily be able to recompile the application
+      to use the modified definitions.)
+
+      b) Use a suitable shared library mechanism for linking with the
+      Library.  A suitable mechanism is one that (1) uses at run time a
+      copy of the library already present on the user's computer system,
+      rather than copying library functions into the executable, and (2)
+      will operate properly with a modified version of the library, if
+      the user installs one, as long as the modified version is
+      interface-compatible with the version that the work was made with.
+
+      c) Accompany the work with a written offer, valid for at
+      least three years, to give the same user the materials
+      specified in Subsection 6a, above, for a charge no more
+      than the cost of performing this distribution.
+
+      d) If distribution of the work is made by offering access to copy
+      from a designated place, offer equivalent access to copy the above
+      specified materials from the same place.
+
+      e) Verify that the user has already received a copy of these
+      materials or that you have already sent this user a copy.
+
+    For an executable, the required form of the "work that uses the
+  Library" must include any data and utility programs needed for
+  reproducing the executable from it.  However, as a special exception,
+  the materials to be distributed need not include anything that is
+  normally distributed (in either source or binary form) with the major
+  components (compiler, kernel, and so on) of the operating system on
+  which the executable runs, unless that component itself accompanies
+  the executable.
+
+    It may happen that this requirement contradicts the license
+  restrictions of other proprietary libraries that do not normally
+  accompany the operating system.  Such a contradiction means you cannot
+  use both them and the Library together in an executable that you
+  distribute.
+
+    7. You may place library facilities that are a work based on the
+  Library side-by-side in a single library together with other library
+  facilities not covered by this License, and distribute such a combined
+  library, provided that the separate distribution of the work based on
+  the Library and of the other library facilities is otherwise
+  permitted, and provided that you do these two things:
+
+      a) Accompany the combined library with a copy of the same work
+      based on the Library, uncombined with any other library
+      facilities.  This must be distributed under the terms of the
+      Sections above.
+
+      b) Give prominent notice with the combined library of the fact
+      that part of it is a work based on the Library, and explaining
+      where to find the accompanying uncombined form of the same work.
+
+    8. You may not copy, modify, sublicense, link with, or distribute
+  the Library except as expressly provided under this License.  Any
+  attempt otherwise to copy, modify, sublicense, link with, or
+  distribute the Library is void, and will automatically terminate your
+  rights under this License.  However, parties who have received copies,
+  or rights, from you under this License will not have their licenses
+  terminated so long as such parties remain in full compliance.
+
+    9. You are not required to accept this License, since you have not
+  signed it.  However, nothing else grants you permission to modify or
+  distribute the Library or its derivative works.  These actions are
+  prohibited by law if you do not accept this License.  Therefore, by
+  modifying or distributing the Library (or any work based on the
+  Library), you indicate your acceptance of this License to do so, and
+  all its terms and conditions for copying, distributing or modifying
+  the Library or works based on it.
+
+    10. Each time you redistribute the Library (or any work based on the
+  Library), the recipient automatically receives a license from the
+  original licensor to copy, distribute, link with or modify the Library
+  subject to these terms and conditions.  You may not impose any further
+  restrictions on the recipients' exercise of the rights granted herein.
+  You are not responsible for enforcing compliance by third parties with
+  this License.
+
+    11. If, as a consequence of a court judgment or allegation of patent
+  infringement or for any other reason (not limited to patent issues),
+  conditions are imposed on you (whether by court order, agreement or
+  otherwise) that contradict the conditions of this License, they do not
+  excuse you from the conditions of this License.  If you cannot
+  distribute so as to satisfy simultaneously your obligations under this
+  License and any other pertinent obligations, then as a consequence you
+  may not distribute the Library at all.  For example, if a patent
+  license would not permit royalty-free redistribution of the Library by
+  all those who receive copies directly or indirectly through you, then
+  the only way you could satisfy both it and this License would be to
+  refrain entirely from distribution of the Library.
+
+  If any portion of this section is held invalid or unenforceable under any
+  particular circumstance, the balance of the section is intended to apply,
+  and the section as a whole is intended to apply in other circumstances.
+
+  It is not the purpose of this section to induce you to infringe any
+  patents or other property right claims or to contest validity of any
+  such claims; this section has the sole purpose of protecting the
+  integrity of the free software distribution system which is
+  implemented by public license practices.  Many people have made
+  generous contributions to the wide range of software distributed
+  through that system in reliance on consistent application of that
+  system; it is up to the author/donor to decide if he or she is willing
+  to distribute software through any other system and a licensee cannot
+  impose that choice.
+
+  This section is intended to make thoroughly clear what is believed to
+  be a consequence of the rest of this License.
+
+    12. If the distribution and/or use of the Library is restricted in
+  certain countries either by patents or by copyrighted interfaces, the
+  original copyright holder who places the Library under this License may add
+  an explicit geographical distribution limitation excluding those countries,
+  so that distribution is permitted only in or among countries not thus
+  excluded.  In such case, this License incorporates the limitation as if
+  written in the body of this License.
+
+    13. The Free Software Foundation may publish revised and/or new
+  versions of the Lesser General Public License from time to time.
+  Such new versions will be similar in spirit to the present version,
+  but may differ in detail to address new problems or concerns.
+
+  Each version is given a distinguishing version number.  If the Library
+  specifies a version number of this License which applies to it and
+  "any later version", you have the option of following the terms and
+  conditions either of that version or of any later version published by
+  the Free Software Foundation.  If the Library does not specify a
+  license version number, you may choose any version ever published by
+  the Free Software Foundation.
+
+    14. If you wish to incorporate parts of the Library into other free
+  programs whose distribution conditions are incompatible with these,
+  write to the author to ask for permission.  For software which is
+  copyrighted by the Free Software Foundation, write to the Free
+  Software Foundation; we sometimes make exceptions for this.  Our
+  decision will be guided by the two goals of preserving the free status
+  of all derivatives of our free software and of promoting the sharing
+  and reuse of software generally.
+
+            NO WARRANTY
+
+    15. BECAUSE THE LIBRARY IS LICENSED FREE OF CHARGE, THERE IS NO
+  WARRANTY FOR THE LIBRARY, TO THE EXTENT PERMITTED BY APPLICABLE LAW.
+  EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR
+  OTHER PARTIES PROVIDE THE LIBRARY "AS IS" WITHOUT WARRANTY OF ANY
+  KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE
+  IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
+  PURPOSE.  THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE
+  LIBRARY IS WITH YOU.  SHOULD THE LIBRARY PROVE DEFECTIVE, YOU ASSUME
+  THE COST OF ALL NECESSARY SERVICING, REPAIR OR CORRECTION.
+
+    16. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN
+  WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY
+  AND/OR REDISTRIBUTE THE LIBRARY AS PERMITTED ABOVE, BE LIABLE TO YOU
+  FOR DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR
+  CONSEQUENTIAL DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE
+  LIBRARY (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING
+  RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A
+  FAILURE OF THE LIBRARY TO OPERATE WITH ANY OTHER SOFTWARE), EVEN IF
+  SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH
+  DAMAGES.
+
+           END OF TERMS AND CONDITIONS
+
+             How to Apply These Terms to Your New Libraries
+
+    If you develop a new library, and you want it to be of the greatest
+  possible use to the public, we recommend making it free software that
+  everyone can redistribute and change.  You can do so by permitting
+  redistribution under these terms (or, alternatively, under the terms of the
+  ordinary General Public License).
+
+    To apply these terms, attach the following notices to the library.  It is
+  safest to attach them to the start of each source file to most effectively
+  convey the exclusion of warranty; and each file should have at least the
+  "copyright" line and a pointer to where the full notice is found.
+
+      <one line to give the library's name and a brief idea of what it does.>
+      Copyright (C) <year>  <name of author>
+
+      This library is free software; you can redistribute it and/or
+      modify it under the terms of the GNU Lesser General Public
+      License as published by the Free Software Foundation; either
+      version 2.1 of the License, or (at your option) any later version.
+
+      This library is distributed in the hope that it will be useful,
+      but WITHOUT ANY WARRANTY; without even the implied warranty of
+      MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
+      Lesser General Public License for more details.
+
+      You should have received a copy of the GNU Lesser General Public
+      License along with this library; if not, write to the Free Software
+      Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA  02111-1307  USA
+
+  Also add information on how to contact you by electronic and paper mail.
+
+  You should also get your employer (if you work as a programmer) or your
+  school, if any, to sign a "copyright disclaimer" for the library, if
+  necessary.  Here is a sample; alter the names:
+
+    Yoyodyne, Inc., hereby disclaims all copyright interest in the
+    library `Frob' (a library for tweaking knobs) written by James Random Hacker.
+
+    <signature of Ty Coon>, 1 April 1990
+    Ty Coon, President of Vice
+
+  That's all there is to it!
+
+The following licenses cover code other than JRuby which is included with JRuby.
+
+Licenses listed below include:
+
+* GNU General Public License version 3
+* Apache 2.0 License
+* BSD License
+* Apache Software License Version 1.1
+* MIT License
+
+The complete text of the GNU General Public License version 3 is as follows:
+
+          GNU GENERAL PUBLIC LICENSE
+             Version 3, 29 June 2007
+
+   Copyright (C) 2007 Free Software Foundation, Inc. <http://fsf.org/>
+   Everyone is permitted to copy and distribute verbatim copies
+   of this license document, but changing it is not allowed.
+
+            Preamble
+
+    The GNU General Public License is a free, copyleft license for
+  software and other kinds of works.
+
+    The licenses for most software and other practical works are designed
+  to take away your freedom to share and change the works.  By contrast,
+  the GNU General Public License is intended to guarantee your freedom to
+  share and change all versions of a program--to make sure it remains free
+  software for all its users.  We, the Free Software Foundation, use the
+  GNU General Public License for most of our software; it applies also to
+  any other work released this way by its authors.  You can apply it to
+  your programs, too.
+
+    When we speak of free software, we are referring to freedom, not
+  price.  Our General Public Licenses are designed to make sure that you
+  have the freedom to distribute copies of free software (and charge for
+  them if you wish), that you receive source code or can get it if you
+  want it, that you can change the software or use pieces of it in new
+  free programs, and that you know you can do these things.
+
+    To protect your rights, we need to prevent others from denying you
+  these rights or asking you to surrender the rights.  Therefore, you have
+  certain responsibilities if you distribute copies of the software, or if
+  you modify it: responsibilities to respect the freedom of others.
+
+    For example, if you distribute copies of such a program, whether
+  gratis or for a fee, you must pass on to the recipients the same
+  freedoms that you received.  You must make sure that they, too, receive
+  or can get the source code.  And you must show them these terms so they
+  know their rights.
+
+    Developers that use the GNU GPL protect your rights with two steps:
+  (1) assert copyright on the software, and (2) offer you this License
+  giving you legal permission to copy, distribute and/or modify it.
+
+    For the developers' and authors' protection, the GPL clearly explains
+  that there is no warranty for this free software.  For both users' and
+  authors' sake, the GPL requires that modified versions be marked as
+  changed, so that their problems will not be attributed erroneously to
+  authors of previous versions.
+
+    Some devices are designed to deny users access to install or run
+  modified versions of the software inside them, although the manufacturer
+  can do so.  This is fundamentally incompatible with the aim of
+  protecting users' freedom to change the software.  The systematic
+  pattern of such abuse occurs in the area of products for individuals to
+  use, which is precisely where it is most unacceptable.  Therefore, we
+  have designed this version of the GPL to prohibit the practice for those
+  products.  If such problems arise substantially in other domains, we
+  stand ready to extend this provision to those domains in future versions
+  of the GPL, as needed to protect the freedom of users.
+
+    Finally, every program is threatened constantly by software patents.
+  States should not allow patents to restrict development and use of
+  software on general-purpose computers, but in those that do, we wish to
+  avoid the special danger that patents applied to a free program could
+  make it effectively proprietary.  To prevent this, the GPL assures that
+  patents cannot be used to render the program non-free.
+
+    The precise terms and conditions for copying, distribution and
+  modification follow.
+
+             TERMS AND CONDITIONS
+
+    0. Definitions.
+
+    "This License" refers to version 3 of the GNU General Public License.
+
+    "Copyright" also means copyright-like laws that apply to other kinds of
+  works, such as semiconductor masks.
+ 
+    "The Program" refers to any copyrightable work licensed under this
+  License.  Each licensee is addressed as "you".  "Licensees" and
+  "recipients" may be individuals or organizations.
+
+    To "modify" a work means to copy from or adapt all or part of the work
+  in a fashion requiring copyright permission, other than the making of an
+  exact copy.  The resulting work is called a "modified version" of the
+  earlier work or a work "based on" the earlier work.
+
+    A "covered work" means either the unmodified Program or a work based
+  on the Program.
+
+    To "propagate" a work means to do anything with it that, without
+  permission, would make you directly or secondarily liable for
+  infringement under applicable copyright law, except executing it on a
+  computer or modifying a private copy.  Propagation includes copying,
+  distribution (with or without modification), making available to the
+  public, and in some countries other activities as well.
+
+    To "convey" a work means any kind of propagation that enables other
+  parties to make or receive copies.  Mere interaction with a user through
+  a computer network, with no transfer of a copy, is not conveying.
+
+    An interactive user interface displays "Appropriate Legal Notices"
+  to the extent that it includes a convenient and prominently visible
+  feature that (1) displays an appropriate copyright notice, and (2)
+  tells the user that there is no warranty for the work (except to the
+  extent that warranties are provided), that licensees may convey the
+  work under this License, and how to view a copy of this License.  If
+  the interface presents a list of user commands or options, such as a
+  menu, a prominent item in the list meets this criterion.
+
+    1. Source Code.
+
+    The "source code" for a work means the preferred form of the work
+  for making modifications to it.  "Object code" means any non-source
+  form of a work.
+
+    A "Standard Interface" means an interface that either is an official
+  standard defined by a recognized standards body, or, in the case of
+  interfaces specified for a particular programming language, one that
+  is widely used among developers working in that language.
+
+    The "System Libraries" of an executable work include anything, other
+  than the work as a whole, that (a) is included in the normal form of
+  packaging a Major Component, but which is not part of that Major
+  Component, and (b) serves only to enable use of the work with that
+  Major Component, or to implement a Standard Interface for which an
+  implementation is available to the public in source code form.  A
+  "Major Component", in this context, means a major essential component
+  (kernel, window system, and so on) of the specific operating system
+  (if any) on which the executable work runs, or a compiler used to
+  produce the work, or an object code interpreter used to run it.
+
+    The "Corresponding Source" for a work in object code form means all
+  the source code needed to generate, install, and (for an executable
+  work) run the object code and to modify the work, including scripts to
+  control those activities.  However, it does not include the work's
+  System Libraries, or general-purpose tools or generally available free
+  programs which are used unmodified in performing those activities but
+  which are not part of the work.  For example, Corresponding Source
+  includes interface definition files associated with source files for
+  the work, and the source code for shared libraries and dynamically
+  linked subprograms that the work is specifically designed to require,
+  such as by intimate data communication or control flow between those
+  subprograms and other parts of the work.
+
+    The Corresponding Source need not include anything that users
+  can regenerate automatically from other parts of the Corresponding
+  Source.
+
+    The Corresponding Source for a work in source code form is that
+  same work.
+
+    2. Basic Permissions.
+
+    All rights granted under this License are granted for the term of
+  copyright on the Program, and are irrevocable provided the stated
+  conditions are met.  This License explicitly affirms your unlimited
+  permission to run the unmodified Program.  The output from running a
+  covered work is covered by this License only if the output, given its
+  content, constitutes a covered work.  This License acknowledges your
+  rights of fair use or other equivalent, as provided by copyright law.
+
+    You may make, run and propagate covered works that you do not
+  convey, without conditions so long as your license otherwise remains
+  in force.  You may convey covered works to others for the sole purpose
+  of having them make modifications exclusively for you, or provide you
+  with facilities for running those works, provided that you comply with
+  the terms of this License in conveying all material for which you do
+  not control copyright.  Those thus making or running the covered works
+  for you must do so exclusively on your behalf, under your direction
+  and control, on terms that prohibit them from making any copies of
+  your copyrighted material outside their relationship with you.
+
+    Conveying under any other circumstances is permitted solely under
+  the conditions stated below.  Sublicensing is not allowed; section 10
+  makes it unnecessary.
+
+    3. Protecting Users' Legal Rights From Anti-Circumvention Law.
+
+    No covered work shall be deemed part of an effective technological
+  measure under any applicable law fulfilling obligations under article
+  11 of the WIPO copyright treaty adopted on 20 December 1996, or
+  similar laws prohibiting or restricting circumvention of such
+  measures.
+
+    When you convey a covered work, you waive any legal power to forbid
+  circumvention of technological measures to the extent such circumvention
+  is effected by exercising rights under this License with respect to
+  the covered work, and you disclaim any intention to limit operation or
+  modification of the work as a means of enforcing, against the work's
+  users, your or third parties' legal rights to forbid circumvention of
+  technological measures.
+
+    4. Conveying Verbatim Copies.
+
+    You may convey verbatim copies of the Program's source code as you
+  receive it, in any medium, provided that you conspicuously and
+  appropriately publish on each copy an appropriate copyright notice;
+  keep intact all notices stating that this License and any
+  non-permissive terms added in accord with section 7 apply to the code;
+  keep intact all notices of the absence of any warranty; and give all
+  recipients a copy of this License along with the Program.
+
+    You may charge any price or no price for each copy that you convey,
+  and you may offer support or warranty protection for a fee.
+
+    5. Conveying Modified Source Versions.
+
+    You may convey a work based on the Program, or the modifications to
+  produce it from the Program, in the form of source code under the
+  terms of section 4, provided that you also meet all of these conditions:
+
+      a) The work must carry prominent notices stating that you modified
+      it, and giving a relevant date.
+
+      b) The work must carry prominent notices stating that it is
+      released under this License and any conditions added under section
+      7.  This requirement modifies the requirement in section 4 to
+      "keep intact all notices".
+
+      c) You must license the entire work, as a whole, under this
+      License to anyone who comes into possession of a copy.  This
+      License will therefore apply, along with any applicable section 7
+      additional terms, to the whole of the work, and all its parts,
+      regardless of how they are packaged.  This License gives no
+      permission to license the work in any other way, but it does not
+      invalidate such permission if you have separately received it.
+
+      d) If the work has interactive user interfaces, each must display
+      Appropriate Legal Notices; however, if the Program has interactive
+      interfaces that do not display Appropriate Legal Notices, your
+      work need not make them do so.
+
+    A compilation of a covered work with other separate and independent
+  works, which are not by their nature extensions of the covered work,
+  and which are not combined with it such as to form a larger program,
+  in or on a volume of a storage or distribution medium, is called an
+  "aggregate" if the compilation and its resulting copyright are not
+  used to limit the access or legal rights of the compilation's users
+  beyond what the individual works permit.  Inclusion of a covered work
+  in an aggregate does not cause this License to apply to the other
+  parts of the aggregate.
+
+    6. Conveying Non-Source Forms.
+
+    You may convey a covered work in object code form under the terms
+  of sections 4 and 5, provided that you also convey the
+  machine-readable Corresponding Source under the terms of this License,
+  in one of these ways:
+
+      a) Convey the object code in, or embodied in, a physical product
+      (including a physical distribution medium), accompanied by the
+      Corresponding Source fixed on a durable physical medium
+      customarily used for software interchange.
+
+      b) Convey the object code in, or embodied in, a physical product
+      (including a physical distribution medium), accompanied by a
+      written offer, valid for at least three years and valid for as
+      long as you offer spare parts or customer support for that product
+      model, to give anyone who possesses the object code either (1) a
+      copy of the Corresponding Source for all the software in the
+      product that is covered by this License, on a durable physical
+      medium customarily used for software interchange, for a price no
+      more than your reasonable cost of physically performing this
+      conveying of source, or (2) access to copy the
+      Corresponding Source from a network server at no charge.
+
+      c) Convey individual copies of the object code with a copy of the
+      written offer to provide the Corresponding Source.  This
+      alternative is allowed only occasionally and noncommercially, and
+      only if you received the object code with such an offer, in accord
+      with subsection 6b.
+
+      d) Convey the object code by offering access from a designated
+      place (gratis or for a charge), and offer equivalent access to the
+      Corresponding Source in the same way through the same place at no
+      further charge.  You need not require recipients to copy the
+      Corresponding Source along with the object code.  If the place to
+      copy the object code is a network server, the Corresponding Source
+      may be on a different server (operated by you or a third party)
+      that supports equivalent copying facilities, provided you maintain
+      clear directions next to the object code saying where to find the
+      Corresponding Source.  Regardless of what server hosts the
+      Corresponding Source, you remain obligated to ensure that it is
+      available for as long as needed to satisfy these requirements.
+
+      e) Convey the object code using peer-to-peer transmission, provided
+      you inform other peers where the object code and Corresponding
+      Source of the work are being offered to the general public at no
+      charge under subsection 6d.
+
+    A separable portion of the object code, whose source code is excluded
+  from the Corresponding Source as a System Library, need not be
+  included in conveying the object code work.
+
+    A "User Product" is either (1) a "consumer product", which means any
+  tangible personal property which is normally used for personal, family,
+  or household purposes, or (2) anything designed or sold for incorporation
+  into a dwelling.  In determining whether a product is a consumer product,
+  doubtful cases shall be resolved in favor of coverage.  For a particular
+  product received by a particular user, "normally used" refers to a
+  typical or common use of that class of product, regardless of the status
+  of the particular user or of the way in which the particular user
+  actually uses, or expects or is expected to use, the product.  A product
+  is a consumer product regardless of whether the product has substantial
+  commercial, industrial or non-consumer uses, unless such uses represent
+  the only significant mode of use of the product.
+
+    "Installation Information" for a User Product means any methods,
+  procedures, authorization keys, or other information required to install
+  and execute modified versions of a covered work in that User Product from
+  a modified version of its Corresponding Source.  The information must
+  suffice to ensure that the continued functioning of the modified object
+  code is in no case prevented or interfered with solely because
+  modification has been made.
+
+    If you convey an object code work under this section in, or with, or
+  specifically for use in, a User Product, and the conveying occurs as
+  part of a transaction in which the right of possession and use of the
+  User Product is transferred to the recipient in perpetuity or for a
+  fixed term (regardless of how the transaction is characterized), the
+  Corresponding Source conveyed under this section must be accompanied
+  by the Installation Information.  But this requirement does not apply
+  if neither you nor any third party retains the ability to install
+  modified object code on the User Product (for example, the work has
+  been installed in ROM).
+
+    The requirement to provide Installation Information does not include a
+  requirement to continue to provide support service, warranty, or updates
+  for a work that has been modified or installed by the recipient, or for
+  the User Product in which it has been modified or installed.  Access to a
+  network may be denied when the modification itself materially and
+  adversely affects the operation of the network or violates the rules and
+  protocols for communication across the network.
+
+    Corresponding Source conveyed, and Installation Information provided,
+  in accord with this section must be in a format that is publicly
+  documented (and with an implementation available to the public in
+  source code form), and must require no special password or key for
+  unpacking, reading or copying.
+
+    7. Additional Terms.
+
+    "Additional permissions" are terms that supplement the terms of this
+  License by making exceptions from one or more of its conditions.
+  Additional permissions that are applicable to the entire Program shall
+  be treated as though they were included in this License, to the extent
+  that they are valid under applicable law.  If additional permissions
+  apply only to part of the Program, that part may be used separately
+  under those permissions, but the entire Program remains governed by
+  this License without regard to the additional permissions.
+
+    When you convey a copy of a covered work, you may at your option
+  remove any additional permissions from that copy, or from any part of
+  it.  (Additional permissions may be written to require their own
+  removal in certain cases when you modify the work.)  You may place
+  additional permissions on material, added by you to a covered work,
+  for which you have or can give appropriate copyright permission.
+
+    Notwithstanding any other provision of this License, for material you
+  add to a covered work, you may (if authorized by the copyright holders of
+  that material) supplement the terms

<TRUNCATED>

[08/50] [abbrv] ignite git commit: IGNITE-10381 Fixed U.doInParallel not to terminate early - Fixes #5483.

Posted by ag...@apache.org.
IGNITE-10381 Fixed U.doInParallel not to terminate early - Fixes #5483.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-9720
Commit: 97d242263a2b2f8f2c6677df1007a6bd3188522b
Parents: ae5bfad
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Fri Nov 23 19:11:31 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Nov 23 19:11:31 2018 +0300

----------------------------------------------------------------------
 .../GridCachePartitionExchangeManager.java      |   4 +-
 .../GridDhtPartitionsExchangeFuture.java        |   3 +-
 .../ignite/internal/util/IgniteUtils.java       | 189 +++----------------
 ...ynamicCacheStartFailWithPersistenceTest.java |   1 +
 .../internal/util/IgniteUtilsSelfTest.java      |  43 -----
 5 files changed, 34 insertions(+), 206 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/97d24226/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index fed5686..6dad367 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -2906,8 +2906,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                                             "topVer=" + exchFut.initialVersion() +
                                             ", node=" + cctx.localNodeId() + "]. " +
                                             (curTimeout <= 0 && !txRolledBack ? "Consider changing " +
-                                                    "TransactionConfiguration.txTimeoutOnPartitionMapSynchronization" +
-                                                    " to non default value to avoid this message. " : "") +
+                                            "TransactionConfiguration.txTimeoutOnPartitionMapSynchronization" +
+                                            " to non default value to avoid this message. " : "") +
                                             "Dumping pending objects that might be the cause: ");
 
                                         try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/97d24226/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 9d81405..3384bb9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -1189,7 +1189,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                 // This exception will be handled by init() method.
                 throw e;
 
-            U.error(log, "Failed to initialize cache(s) (will try to rollback). " + exchId, e);
+            U.error(log, "Failed to initialize cache(s) (will try to rollback) [exchId=" + exchId +
+                ", caches=" + exchActions.cacheGroupsToStart() + ']', e);
 
             exchangeLocE = new IgniteCheckedException(
                 "Failed to initialize exchange locally [locNodeId=" + cctx.localNodeId() + "]", e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/97d24226/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index f1f4253..2d6b584 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -202,7 +202,6 @@ import java.sql.SQLException;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Calendar;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
@@ -221,7 +220,6 @@ import java.util.Random;
 import java.util.ServiceLoader;
 import java.util.Set;
 import java.util.StringTokenizer;
-import java.util.TimeZone;
 import java.util.TreeMap;
 import java.util.UUID;
 import java.util.concurrent.BrokenBarrierException;
@@ -7193,137 +7191,6 @@ public abstract class IgniteUtils {
     }
 
     /**
-     *
-     * @param str ISO date.
-     * @return Calendar instance.
-     * @throws IgniteCheckedException Thrown in case of any errors.
-     */
-    public static Calendar parseIsoDate(String str) throws IgniteCheckedException {
-        StringTokenizer t = new StringTokenizer(str, "+-:.TZ", true);
-
-        Calendar cal = Calendar.getInstance();
-        cal.clear();
-
-        try {
-            if (t.hasMoreTokens())
-                cal.set(Calendar.YEAR, Integer.parseInt(t.nextToken()));
-            else
-                return cal;
-
-            if (checkNextToken(t, "-", str) && t.hasMoreTokens())
-                cal.set(Calendar.MONTH, Integer.parseInt(t.nextToken()) - 1);
-            else
-                return cal;
-
-            if (checkNextToken(t, "-", str) && t.hasMoreTokens())
-                cal.set(Calendar.DAY_OF_MONTH, Integer.parseInt(t.nextToken()));
-            else
-                return cal;
-
-            if (checkNextToken(t, "T", str) && t.hasMoreTokens())
-                cal.set(Calendar.HOUR_OF_DAY, Integer.parseInt(t.nextToken()));
-            else {
-                cal.set(Calendar.HOUR_OF_DAY, 0);
-                cal.set(Calendar.MINUTE, 0);
-                cal.set(Calendar.SECOND, 0);
-                cal.set(Calendar.MILLISECOND, 0);
-
-                return cal;
-            }
-
-            if (checkNextToken(t, ":", str) && t.hasMoreTokens())
-                cal.set(Calendar.MINUTE, Integer.parseInt(t.nextToken()));
-            else {
-                cal.set(Calendar.MINUTE, 0);
-                cal.set(Calendar.SECOND, 0);
-                cal.set(Calendar.MILLISECOND, 0);
-
-                return cal;
-            }
-
-            if (!t.hasMoreTokens())
-                return cal;
-
-            String tok = t.nextToken();
-
-            if (":".equals(tok)) { // Seconds.
-                if (t.hasMoreTokens()) {
-                    cal.set(Calendar.SECOND, Integer.parseInt(t.nextToken()));
-
-                    if (!t.hasMoreTokens())
-                        return cal;
-
-                    tok = t.nextToken();
-
-                    if (".".equals(tok)) {
-                        String nt = t.nextToken();
-
-                        while (nt.length() < 3)
-                            nt += "0";
-
-                        nt = nt.substring(0, 3); // Cut trailing chars.
-
-                        cal.set(Calendar.MILLISECOND, Integer.parseInt(nt));
-
-                        if (!t.hasMoreTokens())
-                            return cal;
-
-                        tok = t.nextToken();
-                    }
-                    else
-                        cal.set(Calendar.MILLISECOND, 0);
-                }
-                else
-                    throw new IgniteCheckedException("Invalid date format: " + str);
-            }
-            else {
-                cal.set(Calendar.SECOND, 0);
-                cal.set(Calendar.MILLISECOND, 0);
-            }
-
-            if (!"Z".equals(tok)) {
-                if (!"+".equals(tok) && !"-".equals(tok))
-                    throw new IgniteCheckedException("Invalid date format: " + str);
-
-                boolean plus = "+".equals(tok);
-
-                if (!t.hasMoreTokens())
-                    throw new IgniteCheckedException("Invalid date format: " + str);
-
-                tok = t.nextToken();
-
-                int tzHour;
-                int tzMin;
-
-                if (tok.length() == 4) {
-                    tzHour = Integer.parseInt(tok.substring(0, 2));
-                    tzMin = Integer.parseInt(tok.substring(2, 4));
-                }
-                else {
-                    tzHour = Integer.parseInt(tok);
-
-                    if (checkNextToken(t, ":", str) && t.hasMoreTokens())
-                        tzMin = Integer.parseInt(t.nextToken());
-                    else
-                        throw new IgniteCheckedException("Invalid date format: " + str);
-                }
-
-                if (plus)
-                    cal.set(Calendar.ZONE_OFFSET, (tzHour * 60 + tzMin) * 60 * 1000);
-                else
-                    cal.set(Calendar.ZONE_OFFSET, -(tzHour * 60 + tzMin) * 60 * 1000);
-            }
-            else
-                cal.setTimeZone(TimeZone.getTimeZone("GMT"));
-        }
-        catch (NumberFormatException ex) {
-            throw new IgniteCheckedException("Invalid date format: " + str, ex);
-        }
-
-        return cal;
-    }
-
-    /**
      * Adds values to collection and returns the same collection to allow chaining.
      *
      * @param c Collection to add values to.
@@ -7429,20 +7296,6 @@ public abstract class IgniteUtils {
     }
 
     /**
-     * Parses passed string with specified date.
-     *
-     * @param src String to parse.
-     * @param ptrn Pattern.
-     * @return Parsed date.
-     * @throws java.text.ParseException If exception occurs while parsing.
-     */
-    public static Date parse(String src, String ptrn) throws java.text.ParseException {
-        java.text.DateFormat format = new java.text.SimpleDateFormat(ptrn);
-
-        return format.parse(src);
-    }
-
-    /**
      * Checks if class loader is an internal P2P class loader.
      *
      * @param o Object to check.
@@ -10841,7 +10694,7 @@ public abstract class IgniteUtils {
 
                 batch.result(res);
             }
-            catch (IgniteCheckedException e) {
+            catch (Throwable e) {
                 batch.result(e);
             }
         }
@@ -10854,10 +10707,7 @@ public abstract class IgniteUtils {
                 Throwable err = batch.error;
 
                 if (err != null) {
-                    if (error == null)
-                        error = err;
-                    else
-                        error.addSuppressed(err);
+                    error = addSuppressed(error, err);
 
                     continue;
                 }
@@ -10875,16 +10725,10 @@ public abstract class IgniteUtils {
                 throw new IgniteInterruptedCheckedException(e);
             }
             catch (ExecutionException e) {
-                if(error == null)
-                    error = e.getCause();
-                else
-                    error.addSuppressed(e.getCause());
+                error = addSuppressed(error, e.getCause());
             }
             catch (CancellationException e) {
-                if(error == null)
-                    error = e;
-                else
-                    error.addSuppressed(e);
+                error = addSuppressed(error, e);
             }
         }
 
@@ -10905,6 +10749,31 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * Utility method to add the given throwable error to the given throwable root error. If the given
+     * suppressed throwable is an {@code Error}, but the root error is not, will change the root to the {@code Error}.
+     *
+     * @param root Root error to add suppressed error to.
+     * @param err Error to add.
+     * @return New root error.
+     */
+    private static Throwable addSuppressed(Throwable root, Throwable err) {
+        assert err != null;
+
+        if (root == null)
+            return err;
+
+        if (err instanceof Error && !(root instanceof Error)) {
+            err.addSuppressed(root);
+
+            root = err;
+        }
+        else
+            root.addSuppressed(err);
+
+        return root;
+    }
+
+    /**
      * The batch of tasks with a batch index in global array.
      */
     private static class Batch<T,R> {

http://git-wip-us.apache.org/repos/asf/ignite/blob/97d24226/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailWithPersistenceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailWithPersistenceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailWithPersistenceTest.java
index 24c9342..8643b66 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailWithPersistenceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartFailWithPersistenceTest.java
@@ -34,6 +34,7 @@ public class IgniteDynamicCacheStartFailWithPersistenceTest extends IgniteAbstra
         return 5 * 60 * 1000;
     }
 
+    /** {@inheritDoc} */
     @Override protected boolean persistenceEnabled() {
         return true;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/97d24226/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java
index 39bb21ea..310705b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java
@@ -38,7 +38,6 @@ import java.net.URL;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Calendar;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
@@ -401,48 +400,6 @@ public class IgniteUtilsSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     *
-     * @throws Exception If failed.
-     */
-    public void testParseIsoDate() throws Exception {
-        Calendar cal = U.parseIsoDate("2009-12-08T13:30:44.000Z");
-
-        assert cal.get(Calendar.YEAR) == 2009;
-        assert cal.get(Calendar.MONTH) == 11;
-        assert cal.get(Calendar.DAY_OF_MONTH) == 8;
-        assert cal.get(Calendar.HOUR_OF_DAY) == 13;
-        assert cal.get(Calendar.MINUTE) == 30;
-        assert cal.get(Calendar.SECOND) == 44;
-        assert cal.get(Calendar.MILLISECOND) == 0;
-        assert cal.get(Calendar.ZONE_OFFSET) == 0 :
-            "Unexpected value: " + cal.get(Calendar.ZONE_OFFSET);
-
-        cal = U.parseIsoDate("2009-12-08T13:30:44.000+03:00");
-
-        assert cal.get(Calendar.YEAR) == 2009;
-        assert cal.get(Calendar.MONTH) == 11;
-        assert cal.get(Calendar.DAY_OF_MONTH) == 8;
-        assert cal.get(Calendar.HOUR_OF_DAY) == 13;
-        assert cal.get(Calendar.MINUTE) == 30;
-        assert cal.get(Calendar.SECOND) == 44;
-        assert cal.get(Calendar.MILLISECOND) == 0;
-        assert cal.get(Calendar.ZONE_OFFSET) == 3 * 60 * 60 * 1000 :
-            "Unexpected value: " + cal.get(Calendar.ZONE_OFFSET);
-
-        cal = U.parseIsoDate("2009-12-08T13:30:44.000+0300");
-
-        assert cal.get(Calendar.YEAR) == 2009;
-        assert cal.get(Calendar.MONTH) == 11;
-        assert cal.get(Calendar.DAY_OF_MONTH) == 8;
-        assert cal.get(Calendar.HOUR_OF_DAY) == 13;
-        assert cal.get(Calendar.MINUTE) == 30;
-        assert cal.get(Calendar.SECOND) == 44;
-        assert cal.get(Calendar.MILLISECOND) == 0;
-        assert cal.get(Calendar.ZONE_OFFSET) == 3 * 60 * 60 * 1000 :
-            "Unexpected value: " + cal.get(Calendar.ZONE_OFFSET);
-    }
-
-    /**
      * @throws Exception If test failed.
      */
     public void testPeerDeployAware0() throws Exception {


[31/50] [abbrv] ignite git commit: IGNITE-10343 Stop grid in IgniteCacheGetRestartTest.testGetRestartReplicated test - Fixes #5447.

Posted by ag...@apache.org.
IGNITE-10343 Stop grid in IgniteCacheGetRestartTest.testGetRestartReplicated test - Fixes #5447.

Signed-off-by: Aleksey Plekhanov <pl...@gmail.com>


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

Branch: refs/heads/ignite-9720
Commit: 46a84fddb701f8e26741ddaa1ee3ab3ffa2234c2
Parents: 7577c87
Author: Fedotov <va...@gmail.com>
Authored: Tue Nov 27 13:24:50 2018 +0300
Committer: Aleksey Plekhanov <pl...@gmail.com>
Committed: Tue Nov 27 13:24:50 2018 +0300

----------------------------------------------------------------------
 .../distributed/IgniteCacheGetRestartTest.java  | 29 +++++++++++---------
 1 file changed, 16 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/46a84fdd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheGetRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheGetRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheGetRestartTest.java
index e194e28..ba22048 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheGetRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheGetRestartTest.java
@@ -209,25 +209,28 @@ public class IgniteCacheGetRestartTest extends GridCommonAbstractTest {
 
                         log.info("Restart node [node=" + nodeIdx + ", client=" + clientMode + ']');
 
-                        Ignite ignite = startGrid(nodeIdx);
+                        try {
+                            Ignite ignite = startGrid(nodeIdx);
 
-                        IgniteCache<Object, Object> cache;
+                            IgniteCache<Object, Object> cache;
 
-                        if (clientMode && ccfg.getNearConfiguration() != null)
-                            cache = ignite.createNearCache(ccfg.getName(), new NearCacheConfiguration<>());
-                        else
-                            cache = ignite.cache(ccfg.getName());
+                            if (clientMode && ccfg.getNearConfiguration() != null)
+                                cache = ignite.createNearCache(ccfg.getName(), new NearCacheConfiguration<>());
+                            else
+                                cache = ignite.cache(ccfg.getName());
 
-                        checkGet(cache);
-
-                        IgniteInternalFuture<?> syncFut = ((IgniteCacheProxy)cache).context().preloader().syncFuture();
-
-                        while (!syncFut.isDone() && U.currentTimeMillis() < stopTime)
                             checkGet(cache);
 
-                        checkGet(cache);
+                            IgniteInternalFuture<?> syncFut = ((IgniteCacheProxy)cache).context().preloader().syncFuture();
+
+                            while (!syncFut.isDone() && U.currentTimeMillis() < stopTime)
+                                checkGet(cache);
 
-                        stopGrid(nodeIdx);
+                            checkGet(cache);
+                        }
+                        finally {
+                            stopGrid(nodeIdx);
+                        }
                     }
 
                     return null;


[10/50] [abbrv] ignite git commit: IGNITE-10392 Use lasdAffChangedTopVer if DiscoCache for the requested topVer isn't ready - Fixes #5488.

Posted by ag...@apache.org.
IGNITE-10392 Use lasdAffChangedTopVer if DiscoCache for the requested topVer isn't ready - Fixes #5488.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-9720
Commit: 5c01c4199df3ba3a479d4566561335d57adffd0f
Parents: 5b656e2
Author: Ilya Lantukh <il...@gridgain.com>
Authored: Fri Nov 23 20:23:16 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Nov 23 20:23:16 2018 +0300

----------------------------------------------------------------------
 .../internal/managers/discovery/GridDiscoveryManager.java    | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5c01c419/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 476fe7e..0a982b0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -2058,6 +2058,14 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             snap.discoCache : discoCacheHist.get(topVer);
 
         if (cache == null) {
+            AffinityTopologyVersion lastAffChangedTopVer =
+                ctx.cache().context().exchange().lastAffinityChangedTopologyVersion(topVer);
+
+            DiscoCache lastAffChangedDiscoCache = discoCacheHist.get(lastAffChangedTopVer);
+
+            if (lastAffChangedDiscoCache != null)
+                return lastAffChangedDiscoCache;
+
             CacheGroupDescriptor desc = ctx.cache().cacheGroupDescriptors().get(grpId);
 
             throw new IgniteException("Failed to resolve nodes topology [" +


[48/50] [abbrv] ignite git commit: IGNITE-10002 Fixed inspections

Posted by ag...@apache.org.
IGNITE-10002 Fixed inspections


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

Branch: refs/heads/ignite-9720
Commit: 2418d876339f8b79666b361848e6378a3d83b7fd
Parents: d21f86f
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Wed Nov 28 11:10:18 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Nov 28 11:12:09 2018 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/CacheEnumOperationsAbstractTest.java | 2 --
 .../communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java | 1 -
 2 files changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2418d876/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
index 480d959..0cffeb6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
@@ -32,7 +32,6 @@ import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
@@ -40,7 +39,6 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-import static org.junit.Assert.fail;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/2418d876/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
index 12cf4d9..27f7fa3 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
@@ -105,7 +105,6 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
     }
 
     /** */
-    @SuppressWarnings({"deprecation"})
     private class TestListener implements CommunicationListener<Message> {
         /** */
         private boolean block;


[13/50] [abbrv] ignite git commit: IGNITE-9145:[ML] Add different strategies to index labels in StringEncoderTrainer

Posted by ag...@apache.org.
IGNITE-9145:[ML] Add different strategies to index labels in
StringEncoderTrainer

this closes #5481


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

Branch: refs/heads/ignite-9720
Commit: 9137af73ef20228ee98e4bc95a8ccb15dadd0010
Parents: cdaeda1
Author: zaleslaw <za...@gmail.com>
Authored: Mon Nov 26 14:10:51 2018 +0300
Committer: Yury Babak <yb...@gridgain.com>
Committed: Mon Nov 26 14:10:51 2018 +0300

----------------------------------------------------------------------
 .../encoding/EncoderSortingStrategy.java        | 31 ++++++++++++++++++++
 .../preprocessing/encoding/EncoderTrainer.java  | 25 +++++++++++++++-
 .../encoding/EncoderTrainerTest.java            | 27 +++++++++++++++++
 3 files changed, 82 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9137af73/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/encoding/EncoderSortingStrategy.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/encoding/EncoderSortingStrategy.java b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/encoding/EncoderSortingStrategy.java
new file mode 100644
index 0000000..22cca53
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/encoding/EncoderSortingStrategy.java
@@ -0,0 +1,31 @@
+/*
+ * 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.ml.preprocessing.encoding;
+
+/**
+ * Describes Encoder sorting strategy to define mapping of integer values to values of categorical feature .
+ *
+ * @see EncoderTrainer
+ */
+public enum EncoderSortingStrategy {
+    /** Descending order by label frequency (most frequent label assigned 0). */
+    FREQUENCY_DESC,
+
+    /** Ascending order by label frequency (least frequent label assigned 0). */
+    FREQUENCY_ASC
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9137af73/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/encoding/EncoderTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/encoding/EncoderTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/encoding/EncoderTrainer.java
index 9a97a6d..d5668e4 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/encoding/EncoderTrainer.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/encoding/EncoderTrainer.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.ml.preprocessing.encoding;
 
+import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
@@ -47,6 +49,9 @@ public class EncoderTrainer<K, V> implements PreprocessingTrainer<K, V, Object[]
     /** Encoder preprocessor type. */
     private EncoderType encoderType = EncoderType.ONE_HOT_ENCODER;
 
+    /** Encoder sorting strategy. */
+    private EncoderSortingStrategy encoderSortingStgy = EncoderSortingStrategy.FREQUENCY_DESC;
+
     /** {@inheritDoc} */
     @Override public EncoderPreprocessor<K, V> fit(DatasetBuilder<K, V> datasetBuilder,
                                                    IgniteBiFunction<K, V, Object[]> basePreprocessor) {
@@ -129,9 +134,16 @@ public class EncoderTrainer<K, V> implements PreprocessingTrainer<K, V, Object[]
      * @return Encoding values.
      */
     private Map<String, Integer> transformFrequenciesToEncodingValues(Map<String, Integer> frequencies) {
+        Comparator<Map.Entry<String, Integer>> comp;
+
+        if (encoderSortingStgy.equals(EncoderSortingStrategy.FREQUENCY_DESC))
+            comp = Map.Entry.comparingByValue();
+        else
+            comp = Collections.reverseOrder(Map.Entry.comparingByValue());
+
         final HashMap<String, Integer> resMap = frequencies.entrySet()
             .stream()
-            .sorted(Map.Entry.comparingByValue())
+            .sorted(comp)
             .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue,
                 (oldValue, newValue) -> oldValue, LinkedHashMap::new));
 
@@ -211,6 +223,17 @@ public class EncoderTrainer<K, V> implements PreprocessingTrainer<K, V, Object[]
     }
 
     /**
+     * Sets the encoder indexing strategy.
+     *
+     * @param encoderSortingStgy The encoder indexing strategy.
+     * @return The changed trainer.
+     */
+    public EncoderTrainer<K, V> withEncoderIndexingStrategy(EncoderSortingStrategy encoderSortingStgy) {
+        this.encoderSortingStgy = encoderSortingStgy;
+        return this;
+    }
+
+    /**
      * Sets the encoder preprocessor type.
      *
      * @param type The encoder preprocessor type.

http://git-wip-us.apache.org/repos/asf/ignite/blob/9137af73/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/encoding/EncoderTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/encoding/EncoderTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/encoding/EncoderTrainerTest.java
index 23afd30..7c7eabe 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/encoding/EncoderTrainerTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/encoding/EncoderTrainerTest.java
@@ -115,4 +115,31 @@ public class EncoderTrainerTest extends TrainerTest {
         }
         fail("UnknownCategorialFeatureValue");
     }
+
+    /** Tests {@code fit()} method. */
+    @Test
+    public void testFitOnStringCategorialFeaturesWithReversedOrder() {
+        Map<Integer, String[]> data = new HashMap<>();
+        data.put(1, new String[] {"Monday", "September"});
+        data.put(2, new String[] {"Monday", "August"});
+        data.put(3, new String[] {"Monday", "August"});
+        data.put(4, new String[] {"Friday", "June"});
+        data.put(5, new String[] {"Friday", "June"});
+        data.put(6, new String[] {"Sunday", "August"});
+
+        DatasetBuilder<Integer, String[]> datasetBuilder = new LocalDatasetBuilder<>(data, parts);
+
+        EncoderTrainer<Integer, String[]> strEncoderTrainer = new EncoderTrainer<Integer, String[]>()
+            .withEncoderType(EncoderType.STRING_ENCODER)
+            .withEncoderIndexingStrategy(EncoderSortingStrategy.FREQUENCY_ASC)
+            .withEncodedFeature(0)
+            .withEncodedFeature(1);
+
+        EncoderPreprocessor<Integer, String[]> preprocessor = strEncoderTrainer.fit(
+            datasetBuilder,
+            (k, v) -> v
+        );
+
+        assertArrayEquals(new double[] {2.0, 0.0}, preprocessor.apply(7, new String[] {"Monday", "September"}).asArray(), 1e-8);
+    }
 }


[16/50] [abbrv] ignite git commit: IGNITE-10330: Disk page compression. - Fixes #5200.

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/compress/src/test/java/org/apache/ignite/internal/processors/compress/FileSystemUtilsTest.java
----------------------------------------------------------------------
diff --git a/modules/compress/src/test/java/org/apache/ignite/internal/processors/compress/FileSystemUtilsTest.java b/modules/compress/src/test/java/org/apache/ignite/internal/processors/compress/FileSystemUtilsTest.java
new file mode 100644
index 0000000..70dda0b
--- /dev/null
+++ b/modules/compress/src/test/java/org/apache/ignite/internal/processors/compress/FileSystemUtilsTest.java
@@ -0,0 +1,193 @@
+/*
+ * 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.internal.processors.compress;
+
+import java.io.FileDescriptor;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import junit.framework.TestCase;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.TRUNCATE_EXISTING;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.compress.CompressionProcessorImpl.allocateDirectBuffer;
+import static org.apache.ignite.internal.processors.compress.FileSystemUtils.getFileSystemBlockSize;
+import static org.apache.ignite.internal.processors.compress.FileSystemUtils.getSparseFileSize;
+import static org.apache.ignite.internal.processors.compress.FileSystemUtils.punchHole;
+
+/**
+ */
+public class FileSystemUtilsTest extends TestCase {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSparseFiles() throws Exception {
+        if (!U.isLinux())
+            return;
+
+        Path file = Files.createTempFile("test_sparse_file_", ".bin");
+
+        try {
+            doTestSparseFiles(file, false); // Ext4 expected as default FS.
+        }
+        finally {
+            Files.delete(file);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void _testFileSystems() throws Exception {
+        doTestSparseFiles(Paths.get("/ext4/test_file"), false);
+        doTestSparseFiles(Paths.get("/btrfs/test_file"), false);
+        doTestSparseFiles(Paths.get("/xfs/test_file"), true);
+    }
+
+    private static int getFD(FileChannel ch) throws IgniteCheckedException {
+        return U.<Integer>field(U.<FileDescriptor>field(ch, "fd"), "fd");
+    }
+
+    /**
+     * @param file File path.
+     * @param reopen Reopen file after each hole punch. XFS needs it.
+     * @throws Exception If failed.
+     */
+    private void doTestSparseFiles(Path file, boolean reopen) throws Exception {
+        System.out.println(file);
+
+        FileChannel ch = FileChannel.open(file,
+            READ, WRITE, TRUNCATE_EXISTING);
+
+        try {
+            int fd = getFD(ch);
+
+            int fsBlockSize = getFileSystemBlockSize(fd);
+
+            System.out.println("fsBlockSize: " + fsBlockSize);
+
+            assertTrue(fsBlockSize > 0);
+
+            int pageSize = fsBlockSize * 4;
+
+            ByteBuffer page = allocateDirectBuffer(pageSize);
+
+            while (page.remaining() > 0)
+                page.putLong(0xABCDEF7654321EADL);
+            page.flip();
+
+            int pages = 5;
+            int blocks = pages * pageSize / fsBlockSize;
+            int fileSize = pages * pageSize;
+            int sparseSize = fileSize;
+
+            for (int i = 0; i < pages; i++) {
+                ch.write(page, i * pageSize);
+                assertEquals(0, page.remaining());
+                page.flip();
+            }
+
+            if (reopen) {
+                ch.force(true);
+                ch.close();
+                ch = FileChannel.open(file, READ, WRITE);
+                fd = getFD(ch);
+            }
+
+            assertEquals(fileSize, ch.size());
+            assertEquals(fileSize, getSparseFileSize(fd));
+
+            int off = fsBlockSize * 3 - (fsBlockSize >>> 2);
+            int len = fsBlockSize;
+            assertEquals(0, punchHole(fd, off, len, fsBlockSize));
+            if (reopen) {
+                ch.force(true);
+                ch.close();
+                ch = FileChannel.open(file, READ, WRITE);
+                fd = getFD(ch);
+            }
+            assertEquals(fileSize, getSparseFileSize(fd));
+
+            off = 2 * fsBlockSize - 3;
+            len = 2 * fsBlockSize + 3;
+            assertEquals(2 * fsBlockSize, punchHole(fd, off, len, fsBlockSize));
+            if (reopen) {
+                ch.force(true);
+                ch.close();
+                ch = FileChannel.open(file, READ, WRITE);
+                fd = getFD(ch);
+            }
+            assertEquals(sparseSize -= 2 * fsBlockSize, getSparseFileSize(fd));
+
+            off = 10 * fsBlockSize;
+            len = 3 * fsBlockSize + 5;
+            assertEquals(3 * fsBlockSize, punchHole(fd, off, len, fsBlockSize));
+            if (reopen) {
+                ch.force(true);
+                ch.close();
+                ch = FileChannel.open(file, READ, WRITE);
+                fd = getFD(ch);
+            }
+            assertEquals(sparseSize -= 3 * fsBlockSize, getSparseFileSize(fd));
+
+            off = 15 * fsBlockSize + 1;
+            len = fsBlockSize;
+            assertEquals(0, punchHole(fd, off, len, fsBlockSize));
+
+            off = 15 * fsBlockSize - 1;
+            len = fsBlockSize;
+            assertEquals(0, punchHole(fd, off, len, fsBlockSize));
+
+            off = 15 * fsBlockSize;
+            len = fsBlockSize - 1;
+            assertEquals(0, punchHole(fd, off, len, fsBlockSize));
+
+            off = 15 * fsBlockSize;
+            len = fsBlockSize;
+            assertEquals(fsBlockSize, punchHole(fd, off, len, fsBlockSize));
+            if (reopen) {
+                ch.force(true);
+                ch.close();
+                ch = FileChannel.open(file, READ, WRITE);
+                fd = getFD(ch);
+            }
+            assertEquals(sparseSize -= fsBlockSize, getSparseFileSize(fd));
+
+            for (int i = 0; i < blocks - 1; i++)
+                punchHole(fd, fsBlockSize * i, fsBlockSize, fsBlockSize);
+
+            if (reopen) {
+                ch.force(true);
+                ch.close();
+                ch = FileChannel.open(file, READ, WRITE);
+                fd = getFD(ch);
+            }
+
+            assertEquals(fsBlockSize, getSparseFileSize(fd));
+        }
+        finally {
+            ch.close();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/compress/src/test/java/org/apache/ignite/testsuites/IgnitePdsCompressionTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/compress/src/test/java/org/apache/ignite/testsuites/IgnitePdsCompressionTestSuite.java b/modules/compress/src/test/java/org/apache/ignite/testsuites/IgnitePdsCompressionTestSuite.java
new file mode 100644
index 0000000..a977700
--- /dev/null
+++ b/modules/compress/src/test/java/org/apache/ignite/testsuites/IgnitePdsCompressionTestSuite.java
@@ -0,0 +1,56 @@
+/*
+ * 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.testsuites;
+
+import junit.framework.TestSuite;
+import org.apache.ignite.internal.processors.compress.CompressionProcessorTest;
+import org.apache.ignite.internal.processors.compress.DiskPageCompressionIntegrationAsyncTest;
+import org.apache.ignite.internal.processors.compress.DiskPageCompressionIntegrationTest;
+import org.apache.ignite.internal.processors.compress.FileSystemUtilsTest;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_DEFAULT_DATA_STORAGE_PAGE_SIZE;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_DEFAULT_DISK_PAGE_COMPRESSION;
+import static org.apache.ignite.configuration.DiskPageCompression.ZSTD;
+
+/**
+ */
+public class IgnitePdsCompressionTestSuite {
+    /**
+     * @return Suite.
+     */
+    public static TestSuite suite() {
+        TestSuite suite = new TestSuite("Ignite Persistent Store Test Suite (with page compression).");
+
+        suite.addTestSuite(CompressionProcessorTest.class);
+        suite.addTestSuite(FileSystemUtilsTest.class);
+        suite.addTestSuite(DiskPageCompressionIntegrationTest.class);
+        suite.addTestSuite(DiskPageCompressionIntegrationAsyncTest.class);
+
+        enableCompressionByDefault();
+        IgnitePdsTestSuite.addRealPageStoreTests(suite);
+
+        return suite;
+    }
+
+    /**
+     */
+    static void enableCompressionByDefault() {
+        System.setProperty(IGNITE_DEFAULT_DISK_PAGE_COMPRESSION, ZSTD.name());
+        System.setProperty(IGNITE_DEFAULT_DATA_STORAGE_PAGE_SIZE, String.valueOf(8 * 1024));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/compress/src/test/java/org/apache/ignite/testsuites/IgnitePdsCompressionTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/compress/src/test/java/org/apache/ignite/testsuites/IgnitePdsCompressionTestSuite2.java b/modules/compress/src/test/java/org/apache/ignite/testsuites/IgnitePdsCompressionTestSuite2.java
new file mode 100644
index 0000000..3fb8ac2
--- /dev/null
+++ b/modules/compress/src/test/java/org/apache/ignite/testsuites/IgnitePdsCompressionTestSuite2.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.testsuites;
+
+import junit.framework.TestSuite;
+
+import static org.apache.ignite.testsuites.IgnitePdsCompressionTestSuite.enableCompressionByDefault;
+
+/**
+ */
+public class IgnitePdsCompressionTestSuite2 {
+    /**
+     * @return Suite.
+     */
+    public static TestSuite suite() {
+        TestSuite suite = new TestSuite("Ignite Persistent Store Test Suite 2 (with page compression).");
+
+        enableCompressionByDefault();
+        IgnitePdsTestSuite2.addRealPageStoreTests(suite);
+
+        return suite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/DataStorageMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/DataStorageMetrics.java b/modules/core/src/main/java/org/apache/ignite/DataStorageMetrics.java
index cdde0ac..e23d188 100644
--- a/modules/core/src/main/java/org/apache/ignite/DataStorageMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/DataStorageMetrics.java
@@ -212,4 +212,22 @@ public interface DataStorageMetrics {
      * @return Checkpoint buffer size in bytes.
      */
     public long getCheckpointBufferSize();
+
+    /**
+     * Storage space allocated in bytes.
+     *
+     * @return Storage space allocated in bytes.
+     */
+    public long getStorageSize();
+
+    /**
+     * Storage space allocated adjusted for possible sparsity in bytes.
+     *
+     * May produce unstable or even incorrect result on some file systems (e.g. XFS).
+     * Known to work correctly on Ext4 and Btrfs.
+     *
+     * @return Storage space allocated adjusted for possible sparsity in bytes
+     *         or negative value is not supported.
+     */
+    public long getSparseStorageSize();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index ccf7ebf..2d27840 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -1047,6 +1047,16 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_RECOVERY_VERBOSE_LOGGING = "IGNITE_RECOVERY_VERBOSE_LOGGING";
 
     /**
+     * Sets default {@link CacheConfiguration#setDiskPageCompression disk page compression}.
+     */
+    public static final String IGNITE_DEFAULT_DISK_PAGE_COMPRESSION = "IGNITE_DEFAULT_DISK_PAGE_COMPRESSION";
+
+    /**
+     * Sets default {@link DataStorageConfiguration#setPageSize storage page size}.
+     */
+    public static final String IGNITE_DEFAULT_DATA_STORAGE_PAGE_SIZE = "IGNITE_DEFAULT_DATA_STORAGE_PAGE_SIZE";
+
+    /**
      * Enforces singleton.
      */
     private IgniteSystemProperties() {
@@ -1054,6 +1064,40 @@ public final class IgniteSystemProperties {
     }
 
     /**
+     * @param enumCls Enum type.
+     * @param name Name of the system property or environment variable.
+     * @return Enum value or {@code null} if the property is not set.
+     */
+    public static <E extends Enum<E>> E getEnum(Class<E> enumCls, String name) {
+        return getEnum(enumCls, name, null);
+    }
+
+    /**
+     * @param name Name of the system property or environment variable.
+     * @return Enum value or the given default.
+     */
+    public static <E extends Enum<E>> E getEnum(String name, E dflt) {
+        return getEnum(dflt.getDeclaringClass(), name, dflt);
+    }
+
+    /**
+     * @param enumCls Enum type.
+     * @param name Name of the system property or environment variable.
+     * @param dflt Default value.
+     * @return Enum value or the given default.
+     */
+    private static <E extends Enum<E>> E getEnum(Class<E> enumCls, String name, E dflt) {
+        assert enumCls != null;
+
+        String val = getString(name);
+
+        if (val == null)
+            return dflt;
+
+        return Enum.valueOf(enumCls, val);
+    }
+
+    /**
      * Gets either system property or environment variable with given name.
      *
      * @param name Name of the system property or environment variable.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 5c91dc0..e27961d 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -33,6 +33,7 @@ import javax.cache.integration.CacheLoader;
 import javax.cache.integration.CacheWriter;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cache.CacheInterceptor;
@@ -62,6 +63,8 @@ import org.apache.ignite.spi.encryption.EncryptionSpi;
 import org.apache.ignite.spi.encryption.keystore.KeystoreEncryptionSpi;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_DEFAULT_DISK_PAGE_COMPRESSION;
+
 /**
  * This class defines grid cache configuration. This configuration is passed to
  * grid via {@link IgniteConfiguration#getCacheConfiguration()} method. It defines all configuration
@@ -383,6 +386,13 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      */
     private boolean encryptionEnabled;
 
+    /** */
+    private DiskPageCompression diskPageCompression = IgniteSystemProperties.getEnum(
+        DiskPageCompression.class, IGNITE_DEFAULT_DISK_PAGE_COMPRESSION);
+
+    /** */
+    private Integer diskPageCompressionLevel;
+
     /** Empty constructor (all values are initialized to their defaults). */
     public CacheConfiguration() {
         /* No-op. */
@@ -443,6 +453,8 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         nearCfg = cc.getNearConfiguration();
         nodeFilter = cc.getNodeFilter();
         onheapCache = cc.isOnheapCacheEnabled();
+        diskPageCompression = cc.getDiskPageCompression();
+        diskPageCompressionLevel = cc.getDiskPageCompressionLevel();
         partLossPlc = cc.getPartitionLossPolicy();
         pluginCfgs = cc.getPluginConfigurations();
         qryDetailMetricsSz = cc.getQueryDetailMetricsSize();
@@ -2297,6 +2309,54 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         return this;
     }
 
+    /**
+     * Gets disk page compression algorithm.
+     * Makes sense only with enabled {@link DataRegionConfiguration#setPersistenceEnabled persistence}.
+     *
+     * @return Disk page compression algorithm.
+     * @see #getDiskPageCompressionLevel
+     */
+    public DiskPageCompression getDiskPageCompression() {
+        return diskPageCompression;
+    }
+
+    /**
+     * Sets disk page compression algorithm.
+     * Makes sense only with enabled {@link DataRegionConfiguration#setPersistenceEnabled persistence}.
+     *
+     * @param diskPageCompression Disk page compression algorithm.
+     * @return {@code this} for chaining.
+     * @see #setDiskPageCompressionLevel
+     */
+    public CacheConfiguration<K,V> setDiskPageCompression(DiskPageCompression diskPageCompression) {
+        this.diskPageCompression = diskPageCompression;
+
+        return this;
+    }
+
+    /**
+     * Gets {@link #getDiskPageCompression algorithm} specific disk page compression level.
+     *
+     * @return Disk page compression level or {@code null} for default.
+     */
+    public Integer getDiskPageCompressionLevel() {
+        return diskPageCompressionLevel;
+    }
+
+    /**
+     * Sets {@link #setDiskPageCompression algorithm} specific disk page compression level.
+     *
+     * @param diskPageCompressionLevel Disk page compression level or {@code null} to use default.
+     *                             {@link DiskPageCompression#ZSTD Zstd}: from {@code -131072} to {@code 22} (default {@code 3}).
+     *                             {@link DiskPageCompression#LZ4 LZ4}: from {@code 0} to {@code 17} (default {@code 0}).
+     * @return {@code this} for chaining.
+     */
+    public CacheConfiguration<K,V> setDiskPageCompressionLevel(Integer diskPageCompressionLevel) {
+        this.diskPageCompressionLevel = diskPageCompressionLevel;
+
+        return this;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(CacheConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java
index 20b314f..4aca0b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java
@@ -28,6 +28,8 @@ import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_DEFAULT_DATA_STORAGE_PAGE_SIZE;
+
 /**
  * A durable memory configuration for an Apache Ignite node. The durable memory is a manageable off-heap based memory
  * architecture that divides all expandable data regions into pages of fixed size
@@ -87,6 +89,12 @@ public class DataStorageConfiguration implements Serializable {
     /** Default memory page size. */
     public static final int DFLT_PAGE_SIZE = 4 * 1024;
 
+    /** Max memory page size. */
+    public static final int MAX_PAGE_SIZE = 16 * 1024;
+
+    /** Min memory page size. */
+    public static final int MIN_PAGE_SIZE = 1024;
+
     /** This name is assigned to default Dataregion if no user-defined default MemPlc is specified */
     public static final String DFLT_DATA_REG_DEFAULT_NAME = "default";
 
@@ -166,7 +174,8 @@ public class DataStorageConfiguration implements Serializable {
     private long sysRegionMaxSize = DFLT_SYS_REG_MAX_SIZE;
 
     /** Memory page size. */
-    private int pageSize;
+    private int pageSize = IgniteSystemProperties.getInteger(
+        IGNITE_DEFAULT_DATA_STORAGE_PAGE_SIZE, 0);
 
     /** Concurrency level. */
     private int concLvl;
@@ -346,10 +355,13 @@ public class DataStorageConfiguration implements Serializable {
      * Changes the page size.
      *
      * @param pageSize Page size in bytes. If value is not set (or zero), {@link #DFLT_PAGE_SIZE} will be used.
+     * @see #MIN_PAGE_SIZE
+     * @see #MAX_PAGE_SIZE
      */
     public DataStorageConfiguration setPageSize(int pageSize) {
         if (pageSize != 0) {
-            A.ensure(pageSize >= 1024 && pageSize <= 16 * 1024, "Page size must be between 1kB and 16kB.");
+            A.ensure(pageSize >= MIN_PAGE_SIZE && pageSize <= MAX_PAGE_SIZE,
+                "Page size must be between 1kB and 16kB.");
             A.ensure(U.isPow2(pageSize), "Page size must be a power of 2.");
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/configuration/DiskPageCompression.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/DiskPageCompression.java b/modules/core/src/main/java/org/apache/ignite/configuration/DiskPageCompression.java
new file mode 100644
index 0000000..d628c6a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/DiskPageCompression.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.configuration;
+
+/**
+ * Disk page compression options.
+ *
+ * @see CacheConfiguration#setDiskPageCompression
+ * @see CacheConfiguration#setDiskPageCompressionLevel
+ */
+public enum DiskPageCompression {
+    /** Retain only useful data from half-filled pages, but do not apply any compression. */
+    SKIP_GARBAGE,
+
+    /** Zstd compression. */
+    ZSTD,
+
+    /** LZ4 compression. */
+    LZ4,
+
+    /** Snappy compression. */
+    SNAPPY
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index a43312c..e19450e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -34,6 +34,7 @@ import org.apache.ignite.internal.managers.failover.GridFailoverManager;
 import org.apache.ignite.internal.managers.indexing.GridIndexingManager;
 import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccProcessor;
+import org.apache.ignite.internal.processors.compress.CompressionProcessor;
 import org.apache.ignite.internal.worker.WorkersRegistry;
 import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
 import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor;
@@ -700,6 +701,11 @@ public interface GridKernalContext extends Iterable<GridComponent> {
     public Thread.UncaughtExceptionHandler uncaughtExceptionHandler();
 
     /**
+     * @return Compression processor.
+     */
+    public CompressionProcessor compress();
+
+    /**
      * @return {@code True} if node is in recovery mode (before join to topology).
      */
     public boolean recoveryMode();

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 8a42664..ef69167 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -49,6 +49,7 @@ import org.apache.ignite.internal.managers.failover.GridFailoverManager;
 import org.apache.ignite.internal.managers.indexing.GridIndexingManager;
 import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccProcessor;
+import org.apache.ignite.internal.processors.compress.CompressionProcessor;
 import org.apache.ignite.internal.worker.WorkersRegistry;
 import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
 import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor;
@@ -294,6 +295,10 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
 
     /** */
     @GridToStringExclude
+    private CompressionProcessor compressProc;
+
+    /** */
+    @GridToStringExclude
     private DataStructuresProcessor dataStructuresProc;
 
     /** Cache mvcc coordinators. */
@@ -639,6 +644,8 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
             internalSubscriptionProc = (GridInternalSubscriptionProcessor)comp;
         else if (comp instanceof IgniteAuthenticationProcessor)
             authProc = (IgniteAuthenticationProcessor)comp;
+        else if (comp instanceof CompressionProcessor)
+            compressProc = (CompressionProcessor)comp;
         else if (!(comp instanceof DiscoveryNodeValidationProcessor
             || comp instanceof PlatformPluginProcessor))
             assert (comp instanceof GridPluginComponent) : "Unknown manager class: " + comp.getClass();
@@ -1184,6 +1191,11 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
+    @Override public CompressionProcessor compress() {
+        return compressProc;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean recoveryMode() {
         return recoveryMode;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java
index 0cd2fc1..65cbb90 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal;
 
 import java.lang.reflect.Constructor;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.compress.CompressionProcessor;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.plugin.extensions.communication.MessageFactory;
 import org.jetbrains.annotations.Nullable;
@@ -89,6 +90,12 @@ public enum IgniteComponentType {
         "org.apache.ignite.internal.processors.schedule.IgniteNoopScheduleProcessor",
         "org.apache.ignite.internal.processors.schedule.IgniteScheduleProcessor",
         "ignite-schedule"
+    ),
+
+    COMPRESSION(
+        CompressionProcessor.class.getName(),
+        "org.apache.ignite.internal.processors.compress.CompressionProcessorImpl",
+        "ignite-compress"
     );
 
     /** No-op class name. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 710fd09..284a4cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -236,6 +236,7 @@ import static org.apache.ignite.internal.GridKernalState.STARTED;
 import static org.apache.ignite.internal.GridKernalState.STARTING;
 import static org.apache.ignite.internal.GridKernalState.STOPPED;
 import static org.apache.ignite.internal.GridKernalState.STOPPING;
+import static org.apache.ignite.internal.IgniteComponentType.COMPRESSION;
 import static org.apache.ignite.internal.IgniteComponentType.HADOOP_HELPER;
 import static org.apache.ignite.internal.IgniteComponentType.IGFS;
 import static org.apache.ignite.internal.IgniteComponentType.IGFS_HELPER;
@@ -1002,6 +1003,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             // Start processors before discovery manager, so they will
             // be able to start receiving messages once discovery completes.
             try {
+                startProcessor(COMPRESSION.createOptional(ctx));
                 startProcessor(new PdsConsistentIdProcessor(ctx));
                 startProcessor(new MvccProcessorImpl(ctx));
                 startProcessor(createComponent(DiscoveryNodeValidationProcessor.class, ctx));

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/PageStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/PageStore.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/PageStore.java
index 7a7f964..7c1e15d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/PageStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/PageStore.java
@@ -17,9 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.store;
 
-import org.apache.ignite.IgniteCheckedException;
-
 import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.persistence.StorageException;
 
 /**
@@ -128,4 +127,35 @@ public interface PageStore {
      * @throws StorageException If failed.
      */
     public void truncate(int tag) throws StorageException;
+
+    /**
+     * @return Page size in bytes.
+     */
+    public int getPageSize();
+
+    /**
+     * @return Storage block size or negative value if unknown or not supported.
+     */
+    public int getBlockSize();
+
+    /**
+     * @return Size of the storage in bytes. May differ from {@link #pages()} * {@link #getPageSize()}
+     *         due to delayed writes or due to other implementation specific details.
+     */
+    public long size();
+
+    /**
+     * @return Size of the storage adjusted for sparsity in bytes or negative
+     *         value if not supported. Should be less than or equal to {@link #size()}.
+     * @see #punchHole
+     */
+    public long getSparseSize();
+
+    /**
+     * Should free all the extra storage space after the given number of useful bytes in the given page.
+     *
+     * @param pageId Page id.
+     * @param usefulBytes Number of meaningful bytes from the beginning of the page.
+     */
+    void punchHole(long pageId, int usefulBytes);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheCompressionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheCompressionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheCompressionManager.java
new file mode 100644
index 0000000..d9e977a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheCompressionManager.java
@@ -0,0 +1,95 @@
+/*
+ * 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.internal.processors.cache;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.DiskPageCompression;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.compress.CompressionProcessor;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.processors.compress.CompressionProcessor.checkCompressionLevelBounds;
+import static org.apache.ignite.internal.processors.compress.CompressionProcessor.getDefaultCompressionLevel;
+
+/**
+ * Cache compression manager.
+ */
+public class CacheCompressionManager extends GridCacheManagerAdapter {
+    /** */
+    private DiskPageCompression diskPageCompression;
+
+    /** */
+    private int diskPageCompressLevel;
+
+    /** */
+    private CompressionProcessor compressProc;
+
+    /** {@inheritDoc} */
+    @Override protected void start0() throws IgniteCheckedException {
+        compressProc = cctx.kernalContext().compress();
+
+        CacheConfiguration cfg = cctx.config();
+
+        diskPageCompression = cfg.getDiskPageCompression();
+
+        if (diskPageCompression != null) {
+            if (!cctx.dataRegion().config().isPersistenceEnabled())
+                throw new IgniteCheckedException("Disk page compression makes sense only with enabled persistence.");
+
+            Integer lvl = cfg.getDiskPageCompressionLevel();
+            diskPageCompressLevel = lvl != null ?
+                checkCompressionLevelBounds(lvl, diskPageCompression) :
+                getDefaultCompressionLevel(diskPageCompression);
+
+            DataStorageConfiguration dsCfg = cctx.kernalContext().config().getDataStorageConfiguration();
+
+            File dbPath = cctx.kernalContext().pdsFolderResolver().resolveFolders().persistentStoreRootPath();
+
+            assert dbPath != null;
+
+            compressProc.checkPageCompressionSupported(dbPath.toPath(), dsCfg.getPageSize());
+
+            if (log.isInfoEnabled()) {
+                log.info("Disk page compression is enabled [cache=" + cctx.name() +
+                    ", compression=" + diskPageCompression + ", level=" + diskPageCompressLevel + "]");
+            }
+        }
+    }
+
+    /**
+     * @param page Page buffer.
+     * @param store Page store.
+     * @return Compressed or the same buffer.
+     * @throws IgniteCheckedException If failed.
+     */
+    public ByteBuffer compressPage(ByteBuffer page, PageStore store) throws IgniteCheckedException {
+        if (diskPageCompression == null)
+            return page;
+
+        int blockSize = store.getBlockSize();
+
+        if (blockSize <= 0)
+            throw new IgniteCheckedException("Failed to detect storage block size on " + U.osString());
+
+        return compressProc.compressPage(page, store.getPageSize(), blockSize, diskPageCompression, diskPageCompressLevel);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsMXBeanImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsMXBeanImpl.java
index 5ece77f..59894e3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsMXBeanImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsMXBeanImpl.java
@@ -28,15 +28,17 @@ import java.util.UUID;
 import java.util.concurrent.atomic.LongAdder;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.pagemem.store.PageStore;
 import org.apache.ignite.internal.processors.affinity.AffinityAssignment;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.persistence.AllocatedPageTracker;
 import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
 import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetricsImpl;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
 import org.apache.ignite.mxbean.CacheGroupMetricsMXBean;
 
 /**
@@ -359,4 +361,21 @@ public class CacheGroupMetricsMXBeanImpl implements CacheGroupMetricsMXBean {
     @Override public long getTotalAllocatedSize() {
         return getTotalAllocatedPages() * ctx.dataRegion().pageMemory().pageSize();
     }
+
+    /** {@inheritDoc} */
+    @Override public long getStorageSize() {
+        return database().forGroupPageStores(ctx, PageStore::size);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSparseStorageSize() {
+        return database().forGroupPageStores(ctx, PageStore::getSparseSize);
+    }
+
+    /**
+     * @return Database.
+     */
+    private GridCacheDatabaseSharedManager database() {
+        return (GridCacheDatabaseSharedManager)ctx.shared().database();
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 30cf969..1a8cf88 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -177,6 +177,9 @@ public class GridCacheContext<K, V> implements Externalizable {
     /** Store manager. */
     private CacheStoreManager storeMgr;
 
+    /** Compression manager. */
+    private CacheCompressionManager compressMgr;
+
     /** Replication manager. */
     private GridCacheDrManager drMgr;
 
@@ -321,6 +324,7 @@ public class GridCacheContext<K, V> implements Externalizable {
          * ===========================
          */
 
+        CacheCompressionManager compressMgr,
         GridCacheEventManager evtMgr,
         CacheStoreManager storeMgr,
         CacheEvictionManager evictMgr,
@@ -338,6 +342,7 @@ public class GridCacheContext<K, V> implements Externalizable {
         assert cacheCfg != null;
         assert locStartTopVer != null : cacheCfg.getName();
 
+        assert compressMgr != null;
         assert grp != null;
         assert evtMgr != null;
         assert storeMgr != null;
@@ -364,6 +369,7 @@ public class GridCacheContext<K, V> implements Externalizable {
          * Managers in starting order!
          * ===========================
          */
+        this.compressMgr = add(compressMgr);
         this.evtMgr = add(evtMgr);
         this.storeMgr = add(storeMgr);
         this.evictMgr = add(evictMgr);
@@ -1230,6 +1236,13 @@ public class GridCacheContext<K, V> implements Externalizable {
     }
 
     /**
+     * @return Compression manager.
+     */
+    public CacheCompressionManager compress() {
+        return compressMgr;
+    }
+
+    /**
      * Sets cache object context.
      *
      * @param cacheObjCtx Cache object context.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index ce81468..8a54852 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -1524,6 +1524,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         boolean nearEnabled = GridCacheUtils.isNearEnabled(cfg);
 
+        CacheCompressionManager compressMgr = new CacheCompressionManager();
         GridCacheAffinityManager affMgr = new GridCacheAffinityManager();
         GridCacheEventManager evtMgr = new GridCacheEventManager();
         CacheEvictionManager evictMgr = (nearEnabled || cfg.isOnheapCacheEnabled()) ? new GridCacheEvictionManager() : new CacheOffheapEvictionManager();
@@ -1558,6 +1559,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
              * Managers in starting order!
              * ===========================
              */
+            compressMgr,
             evtMgr,
             storeMgr,
             evictMgr,
@@ -1694,6 +1696,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                  * Managers in starting order!
                  * ===========================
                  */
+                compressMgr,
                 evtMgr,
                 storeMgr,
                 evictMgr,

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsImpl.java
index 03955a4..4565b58 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsImpl.java
@@ -93,6 +93,12 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
     /** */
     private volatile Collection<DataRegionMetrics> regionMetrics;
 
+    /** */
+    private volatile long storageSize;
+
+    /** */
+    private volatile long sparseStorageSize;
+
     /**
      * @param metricsEnabled Metrics enabled flag.
      * @param rateTimeInterval Rate time interval.
@@ -485,6 +491,16 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
         return metricsEnabled;
     }
 
+    /** {@inheritDoc} */
+    @Override public long getStorageSize() {
+        return storageSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSparseStorageSize() {
+        return sparseStorageSize;
+    }
+
     /**
      * @param lockWaitDuration Lock wait duration.
      * @param markDuration Mark duration.
@@ -503,7 +519,9 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
         long duration,
         long totalPages,
         long dataPages,
-        long cowPages
+        long cowPages,
+        long storageSize,
+        long sparseStorageSize
     ) {
         if (metricsEnabled) {
             lastCpLockWaitDuration = lockWaitDuration;
@@ -514,6 +532,8 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
             lastCpTotalPages = totalPages;
             lastCpDataPages = dataPages;
             lastCpCowPages = cowPages;
+            this.storageSize = storageSize;
+            this.sparseStorageSize = sparseStorageSize;
 
             totalCheckpointTime.addAndGet(duration);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsSnapshot.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsSnapshot.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsSnapshot.java
index c3bcd5b..78b08bc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsSnapshot.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsSnapshot.java
@@ -101,6 +101,12 @@ public class DataStorageMetricsSnapshot implements DataStorageMetrics {
     /** */
     private long totalAllocatedSize;
 
+    /** */
+    private long storageSize;
+
+    /** */
+    private long sparseStorageSize;
+
     /**
      * @param metrics Metrics.
      */
@@ -131,6 +137,8 @@ public class DataStorageMetricsSnapshot implements DataStorageMetrics {
         offHeapSize = metrics.getOffHeapSize();
         offHeadUsedSize = metrics.getOffheapUsedSize();
         totalAllocatedSize = metrics.getTotalAllocatedSize();
+        storageSize = metrics.getStorageSize();
+        sparseStorageSize = metrics.getSparseStorageSize();
     }
 
     /** {@inheritDoc} */
@@ -264,6 +272,16 @@ public class DataStorageMetricsSnapshot implements DataStorageMetrics {
     }
 
     /** {@inheritDoc} */
+    @Override public long getStorageSize() {
+        return storageSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSparseStorageSize() {
+        return sparseStorageSize;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(DataStorageMetricsSnapshot.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index ed54f65..9a083f8 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -54,6 +54,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.LongAdder;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.function.Predicate;
+import java.util.function.ToLongFunction;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
@@ -398,6 +399,13 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                     : ctx.config().getFailureDetectionTimeout()));
     }
 
+    /**
+     * @return File store manager.
+     */
+    public FilePageStoreManager getFileStoreManager() {
+        return storeMgr;
+    }
+
     /** */
     private void notifyMetastorageReadyForRead() throws IgniteCheckedException {
         for (MetastorageLifecycleListener lsnr : metastorageLifecycleLsnrs)
@@ -1935,6 +1943,44 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     }
 
     /**
+     * @param f Consumer.
+     * @return Accumulated result for all page stores.
+     */
+    public long forAllPageStores(ToLongFunction<PageStore> f) {
+        long res = 0;
+
+        for (CacheGroupContext gctx : cctx.cache().cacheGroups())
+            res += forGroupPageStores(gctx, f);
+
+        return res;
+    }
+
+    /**
+     * @param gctx Group context.
+     * @param f Consumer.
+     * @return Accumulated result for all page stores.
+     */
+    public long forGroupPageStores(CacheGroupContext gctx, ToLongFunction<PageStore> f) {
+        int groupId = gctx.groupId();
+
+        long res = 0;
+
+        try {
+            Collection<PageStore> stores = storeMgr.getStores(groupId);
+
+            if (stores != null) {
+                for (PageStore store : stores)
+                    res += f.applyAsLong(store);
+            }
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
+        }
+
+        return res;
+    }
+
+    /**
      * Calculates tail pointer for WAL at the end of logical recovery.
      *
      * @param from Start replay WAL from.
@@ -3263,28 +3309,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                                 tracker.totalDuration()));
                         }
                     }
-
-                    persStoreMetrics.onCheckpoint(
-                        tracker.lockWaitDuration(),
-                        tracker.markDuration(),
-                        tracker.pagesWriteDuration(),
-                        tracker.fsyncDuration(),
-                        tracker.totalDuration(),
-                        chp.pagesSize,
-                        tracker.dataPagesWritten(),
-                        tracker.cowPagesWritten());
-                }
-                else {
-                    persStoreMetrics.onCheckpoint(
-                        tracker.lockWaitDuration(),
-                        tracker.markDuration(),
-                        tracker.pagesWriteDuration(),
-                        tracker.fsyncDuration(),
-                        tracker.totalDuration(),
-                        chp.pagesSize,
-                        tracker.dataPagesWritten(),
-                        tracker.cowPagesWritten());
                 }
+
+                updateMetrics(chp, tracker);
             }
             catch (IgniteCheckedException e) {
                 if (chp != null)
@@ -3294,6 +3321,26 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             }
         }
 
+        /**
+         * @param chp Checkpoint.
+         * @param tracker Tracker.
+         */
+        private void updateMetrics(Checkpoint chp, CheckpointMetricsTracker tracker) {
+            if (persStoreMetrics.metricsEnabled()) {
+                persStoreMetrics.onCheckpoint(
+                    tracker.lockWaitDuration(),
+                    tracker.markDuration(),
+                    tracker.pagesWriteDuration(),
+                    tracker.fsyncDuration(),
+                    tracker.totalDuration(),
+                    chp.pagesSize,
+                    tracker.dataPagesWritten(),
+                    tracker.cowPagesWritten(),
+                    forAllPageStores(PageStore::size),
+                    forAllPageStores(PageStore::getSparseSize));
+            }
+        }
+
         /** */
         private String prepareWalSegsCoveredMsg(IgniteBiTuple<Long, Long> walRange) {
             String res;

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AbstractFileIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AbstractFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AbstractFileIO.java
index d0211f4..d0aaef7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AbstractFileIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AbstractFileIO.java
@@ -56,7 +56,7 @@ public abstract class AbstractFileIO implements FileIO {
                     i += n;
                     time = 0;
                 }
-                else if (n == 0) {
+                else if (n == 0 || i > 0) {
                     if (!write && available(num - i, position + i) == 0)
                         return i;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java
index fd00e25..7c6ece8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache.persistence.file;
 
 import java.io.File;
+import java.io.FileDescriptor;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.MappedByteBuffer;
@@ -25,8 +26,10 @@ import java.nio.channels.AsynchronousFileChannel;
 import java.nio.channels.CompletionHandler;
 import java.nio.file.OpenOption;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.compress.FileSystemUtils;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
 
 /**
  * File I/O implementation based on {@link AsynchronousFileChannel}.
@@ -37,6 +40,12 @@ public class AsyncFileIO extends AbstractFileIO {
      */
     private final AsynchronousFileChannel ch;
 
+    /** Native file descriptor. */
+    private final int fd;
+
+    /** */
+    private final int fsBlockSize;
+
     /**
      * Channel's position.
      */
@@ -54,11 +63,36 @@ public class AsyncFileIO extends AbstractFileIO {
      * @param modes Open modes.
      */
     public AsyncFileIO(File file, ThreadLocal<ChannelOpFuture> holder, OpenOption... modes) throws IOException {
-        this.ch = AsynchronousFileChannel.open(file.toPath(), modes);
-
+        ch = AsynchronousFileChannel.open(file.toPath(), modes);
+        fd = getFileDescriptor(ch);
+        fsBlockSize = FileSystemUtils.getFileSystemBlockSize(fd);
         this.holder = holder;
     }
 
+    /**
+     * @param ch File channel.
+     * @return Native file descriptor.
+     */
+    private static int getFileDescriptor(AsynchronousFileChannel ch) {
+         FileDescriptor fd = U.field(ch, "fdObj");
+         return U.field(fd, "fd");
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getFileSystemBlockSize() {
+        return fsBlockSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSparseSize() {
+        return FileSystemUtils.getSparseFileSize(fd);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int punchHole(long position, int len) {
+        return (int)FileSystemUtils.punchHole(fd, position, len, fsBlockSize);
+    }
+
     /** {@inheritDoc} */
     @Override public long position() throws IOException {
         return position;

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/EncryptedFileIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/EncryptedFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/EncryptedFileIO.java
index 86d9bbc..f21b8ee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/EncryptedFileIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/EncryptedFileIO.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.io.Serializable;
 import java.nio.ByteBuffer;
 import java.nio.MappedByteBuffer;
-
 import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc;
 import org.apache.ignite.spi.encryption.EncryptionSpi;
 import org.apache.ignite.internal.managers.encryption.GridEncryptionManager;
@@ -100,6 +99,21 @@ public class EncryptedFileIO implements FileIO {
     }
 
     /** {@inheritDoc} */
+    @Override public int getFileSystemBlockSize() {
+        return -1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSparseSize() {
+        return -1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int punchHole(long position, int len) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
     @Override public long position() throws IOException {
         return plainFileIO.position();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java
index 6f32d01..546d1a7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java
@@ -253,4 +253,23 @@ public interface FileIO extends AutoCloseable {
      * @throws IOException If some I/O error occurs.
      */
     @Override public void close() throws IOException;
+
+    /**
+     * @return File system block size or negative value if unknown.
+     */
+    public int getFileSystemBlockSize();
+
+    /**
+     * @param position Starting file position.
+     * @param len Number of bytes to free.
+     * @return The actual freed size or negative value if not supported.
+     */
+    int punchHole(long position, int len);
+
+    /**
+     * @return Approximate system dependent size of the storage or negative
+     *          value if not supported.
+     * @see #punchHole
+     */
+    long getSparseSize();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java
index 8e79b54..c615a34 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java
@@ -26,7 +26,7 @@ import java.nio.MappedByteBuffer;
  */
 public class FileIODecorator extends AbstractFileIO {
     /** File I/O delegate */
-    private final FileIO delegate;
+    protected final FileIO delegate;
 
     /**
      *
@@ -37,6 +37,21 @@ public class FileIODecorator extends AbstractFileIO {
     }
 
     /** {@inheritDoc} */
+    @Override public int getFileSystemBlockSize() {
+        return delegate.getFileSystemBlockSize();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSparseSize() {
+        return delegate.getSparseSize();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int punchHole(long pos, int len) {
+        return delegate.punchHole(pos, len);
+    }
+
+    /** {@inheritDoc} */
     @Override public long position() throws IOException {
         return delegate.position();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
index 16d74c3..a8fae08 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
@@ -27,8 +27,8 @@ import java.nio.file.Files;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
@@ -71,7 +71,7 @@ public class FilePageStore implements PageStore {
     private final FileIOFactory ioFactory;
 
     /** I/O interface for read/write operations with file */
-    private volatile FileIO fileIO;
+    protected volatile FileIO fileIO;
 
     /** */
     private final AtomicLong allocated;
@@ -80,7 +80,7 @@ public class FilePageStore implements PageStore {
     private final AllocatedPageTracker allocatedTracker;
 
     /** */
-    private final int pageSize;
+    protected final int pageSize;
 
     /** */
     private volatile boolean inited;
@@ -105,7 +105,8 @@ public class FilePageStore implements PageStore {
         File file,
         FileIOFactory factory,
         DataStorageConfiguration cfg,
-        AllocatedPageTracker allocatedTracker) {
+        AllocatedPageTracker allocatedTracker
+    ) {
         this.type = type;
         this.cfgFile = file;
         this.dbCfg = cfg;
@@ -116,6 +117,38 @@ public class FilePageStore implements PageStore {
     }
 
     /** {@inheritDoc} */
+    @Override public int getPageSize() {
+        return pageSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getBlockSize() {
+        return -1; // Header is unaligned in this version.
+    }
+
+    /** {@inheritDoc} */
+    @Override public long size() {
+        try {
+            FileIO io = fileIO;
+
+            return io == null ? 0 : io.size();
+        }
+        catch (IOException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSparseSize() {
+        return -1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void punchHole(long pageId, int usefulBytes) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean exists() {
         return cfgFile.exists() && cfgFile.length() > headerSize();
     }
@@ -228,10 +261,8 @@ public class FilePageStore implements PageStore {
         if (fileSize == headerSize()) // Every file has a special meta page.
             fileSize = pageSize + headerSize();
 
-        if ((fileSize - headerSize()) % pageSize != 0)
-            throw new IOException(prefix + "(invalid file size)" +
-                " [fileSize=" + U.hexLong(fileSize) +
-                ", pageSize=" + U.hexLong(pageSize) + ']');
+        if (fileSize % pageSize != 0) // In the case of compressed pages we can miss the tail of the page.
+            fileSize = (fileSize / pageSize + 1) * pageSize;
 
         return fileSize;
     }
@@ -333,6 +364,26 @@ public class FilePageStore implements PageStore {
         }
     }
 
+    /**
+     * @param pageId Page ID.
+     * @param pageBuf Page buffer.
+     * @return Number of bytes to calculate CRC on.
+     */
+    private int getCrcSize(long pageId, ByteBuffer pageBuf) throws IOException {
+        int compressedSize = PageIO.getCompressedSize(pageBuf);
+
+        if (compressedSize == 0)
+            return pageSize; // Page is not compressed.
+
+        if (compressedSize < 0 || compressedSize > pageSize) {
+            throw new IgniteDataIntegrityViolationException("Failed to read page (CRC validation failed) " +
+                "[id=" + U.hexLong(pageId) + ", file=" + cfgFile.getAbsolutePath() + ", fileSize=" + fileIO.size() +
+                ", page=" + U.toHexString(pageBuf) + "]");
+        }
+
+        return compressedSize;
+    }
+
     /** {@inheritDoc} */
     @Override public void read(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteCheckedException {
         init();
@@ -363,7 +414,7 @@ public class FilePageStore implements PageStore {
             pageBuf.position(0);
 
             if (!skipCrc) {
-                int curCrc32 = FastCrc.calcCrc(pageBuf, pageSize);
+                int curCrc32 = FastCrc.calcCrc(pageBuf, getCrcSize(pageId, pageBuf));
 
                 if ((savedCrc32 ^ curCrc32) != 0)
                     throw new IgniteDataIntegrityViolationException("Failed to read page (CRC validation failed) " +
@@ -549,7 +600,6 @@ public class FilePageStore implements PageStore {
                         "off=" + U.hexLong(off) + ", allocated=" + U.hexLong(allocated.get()) +
                             ", pageId=" + U.hexLong(pageId) + ", file=" + cfgFile.getPath();
 
-                    assert pageBuf.capacity() == pageSize;
                     assert pageBuf.position() == 0;
                     assert pageBuf.order() == ByteOrder.nativeOrder() : "Page buffer order " + pageBuf.order()
                         + " should be same with " + ByteOrder.nativeOrder();
@@ -559,7 +609,7 @@ public class FilePageStore implements PageStore {
                     if (calculateCrc && !skipCrc) {
                         assert PageIO.getCrc(pageBuf) == 0 : U.hexLong(pageId);
 
-                        PageIO.setCrc(pageBuf, calcCrc32(pageBuf, pageSize));
+                        PageIO.setCrc(pageBuf, calcCrc32(pageBuf, getCrcSize(pageId, pageBuf)));
                     }
 
                     // Check whether crc was calculated somewhere above the stack if it is forcibly skipped.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
index 4a14c6b..86560ba 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
@@ -30,6 +30,7 @@ import java.nio.file.DirectoryStream;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.StandardCopyOption;
+import java.util.AbstractList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -54,6 +55,7 @@ import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
 import org.apache.ignite.internal.pagemem.store.PageStore;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
 import org.apache.ignite.internal.processors.cache.StoredCacheData;
 import org.apache.ignite.internal.processors.cache.persistence.AllocatedPageTracker;
@@ -62,6 +64,7 @@ import org.apache.ignite.internal.processors.cache.persistence.StorageException;
 import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings;
 import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage;
 import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.CU;
@@ -73,6 +76,7 @@ import org.jetbrains.annotations.Nullable;
 
 import static java.nio.file.Files.delete;
 import static java.nio.file.Files.newDirectoryStream;
+import static java.util.Objects.requireNonNull;
 
 /**
  * File page store manager.
@@ -461,6 +465,10 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
 
         try {
             store.read(pageId, pageBuf, keepCrc);
+
+            assert keepCrc || PageIO.getCrc(pageBuf) == 0: store.size() - store.pageOffset(pageId);
+
+            cctx.kernalContext().compress().decompressPage(pageBuf, store.getPageSize());
         }
         catch (StorageException e) {
             cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
@@ -511,13 +519,40 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
      * @return PageStore to which the page has been written.
      * @throws IgniteCheckedException If IO error occurred.
      */
-    public PageStore writeInternal(int cacheId, long pageId, ByteBuffer pageBuf, int tag, boolean calculateCrc) throws IgniteCheckedException {
+    public PageStore writeInternal(int cacheId, long pageId, ByteBuffer pageBuf, int tag, boolean calculateCrc)
+        throws IgniteCheckedException {
         int partId = PageIdUtils.partId(pageId);
 
         PageStore store = getStore(cacheId, partId);
 
         try {
+            int pageSize = store.getPageSize();
+            int compressedPageSize = pageSize;
+
+            GridCacheContext cctx0 = cctx.cacheContext(cacheId);
+
+            if (cctx0 != null) {
+                assert pageBuf.position() == 0 && pageBuf.limit() == pageSize: pageBuf;
+
+                ByteBuffer compressedPageBuf = cctx0.compress().compressPage(pageBuf, store);
+
+                if (compressedPageBuf != pageBuf) {
+                    compressedPageSize = PageIO.getCompressedSize(compressedPageBuf);
+
+                    if (!calculateCrc) {
+                        calculateCrc = true;
+                        PageIO.setCrc(compressedPageBuf, 0); // It will be recalculated over compressed data further.
+                    }
+
+                    PageIO.setCrc(pageBuf, 0); // It is expected to be reset to 0 after each write.
+                    pageBuf = compressedPageBuf;
+                }
+            }
+
             store.write(pageId, pageBuf, tag, calculateCrc);
+
+            if (pageSize > compressedPageSize)
+                store.punchHole(pageId, compressedPageSize); // TODO maybe add async punch mode?
         }
         catch (StorageException e) {
             cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
@@ -1048,6 +1083,15 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
 
     /**
      * @param grpId Cache group ID.
+     * @return Collection of related page stores.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Collection<PageStore> getStores(int grpId) throws IgniteCheckedException {
+        return getHolder(grpId);
+    }
+
+    /**
+     * @param grpId Cache group ID.
      * @param partId Partition ID.
      * @return Page store for the corresponding parameters.
      * @throws IgniteCheckedException If cache or partition with the given ID was not created.
@@ -1125,7 +1169,7 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
     /**
      *
      */
-    private static class CacheStoreHolder {
+    private static class CacheStoreHolder extends AbstractList<PageStore> {
         /** Index store. */
         private final PageStore idxStore;
 
@@ -1133,11 +1177,20 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
         private final PageStore[] partStores;
 
         /**
-         *
          */
-        public CacheStoreHolder(PageStore idxStore, PageStore[] partStores) {
-            this.idxStore = idxStore;
-            this.partStores = partStores;
+        CacheStoreHolder(PageStore idxStore, PageStore[] partStores) {
+            this.idxStore = requireNonNull(idxStore);
+            this.partStores = requireNonNull(partStores);
+        }
+
+        /** {@inheritDoc} */
+        @Override public PageStore get(int idx) {
+            return requireNonNull(idx == partStores.length ? idxStore : partStores[idx]);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int size() {
+            return partStores.length + 1;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreV2.java
index d8c800d..de078eb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreV2.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreV2.java
@@ -57,4 +57,25 @@ public class FilePageStoreV2 extends FilePageStore {
     @Override public int version() {
         return VERSION;
     }
+
+    /** {@inheritDoc} */
+    @Override public int getBlockSize() {
+        return fileIO.getFileSystemBlockSize();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSparseSize() {
+        FileIO io = fileIO;
+
+        return io == null ? 0 : fileIO.getSparseSize();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void punchHole(long pageId, int usefulBytes) {
+        assert usefulBytes >= 0 && usefulBytes < pageSize: usefulBytes;
+
+        long off = pageOffset(pageId);
+
+        fileIO.punchHole(off + usefulBytes, pageSize - usefulBytes);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java
index ef4a3df..c6922bc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java
@@ -18,11 +18,14 @@
 package org.apache.ignite.internal.processors.cache.persistence.file;
 
 import java.io.File;
+import java.io.FileDescriptor;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.MappedByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.file.OpenOption;
+import org.apache.ignite.internal.processors.compress.FileSystemUtils;
+import org.apache.ignite.internal.util.typedef.internal.U;
 
 /**
  * File I/O implementation based on {@link FileChannel}.
@@ -33,6 +36,12 @@ public class RandomAccessFileIO extends AbstractFileIO {
      */
     private final FileChannel ch;
 
+    /** Native file descriptor. */
+    private final int fd;
+
+    /** */
+    private final int fsBlockSize;
+
     /**
      * Creates I/O implementation for specified {@code file}
      *
@@ -41,6 +50,32 @@ public class RandomAccessFileIO extends AbstractFileIO {
      */
     public RandomAccessFileIO(File file, OpenOption... modes) throws IOException {
         ch = FileChannel.open(file.toPath(), modes);
+        fd = getNativeFileDescriptor(ch);
+        fsBlockSize = FileSystemUtils.getFileSystemBlockSize(fd);
+    }
+
+    /**
+     * @param ch File channel.
+     * @return Native file descriptor.
+     */
+    private static int getNativeFileDescriptor(FileChannel ch) {
+        FileDescriptor fd = U.field(ch, "fd");
+        return U.field(fd, "fd");
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getFileSystemBlockSize() {
+        return fsBlockSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSparseSize() {
+        return FileSystemUtils.getSparseFileSize(fd);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int punchHole(long position, int len) {
+        return (int)FileSystemUtils.punchHole(fd, position, len, fsBlockSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/UnzipFileIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/UnzipFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/UnzipFileIO.java
index 6345b1f..5300d83 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/UnzipFileIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/UnzipFileIO.java
@@ -54,6 +54,21 @@ public class UnzipFileIO extends AbstractFileIO {
     }
 
     /** {@inheritDoc} */
+    @Override public int getFileSystemBlockSize() {
+        return -1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSparseSize() {
+        return -1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int punchHole(long position, int len) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
     @Override public long position() throws IOException {
         return totalBytesRead;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
index b64b294..03f66c0 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
@@ -530,7 +530,7 @@ public class PageMemoryImpl implements PageMemoryEx {
             PageHeader.writeTimestamp(absPtr, U.currentTimeMillis());
             rwLock.init(absPtr + PAGE_LOCK_OFFSET, PageIdUtils.tag(pageId));
 
-            assert GridUnsafe.getInt(absPtr + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480
+            assert PageIO.getCrc(absPtr + PAGE_OVERHEAD) == 0; //TODO GG-11480
 
             assert !PageHeader.isAcquired(absPtr) :
                 "Pin counter must be 0 for a new page [relPtr=" + U.hexLong(relPtr) +
@@ -808,9 +808,9 @@ public class PageMemoryImpl implements PageMemoryEx {
 
                     memMetrics.onPageRead();
                 }
-                catch (IgniteDataIntegrityViolationException ignore) {
+                catch (IgniteDataIntegrityViolationException e) {
                     U.warn(log, "Failed to read page (data integrity violation encountered, will try to " +
-                        "restore using existing WAL) [fullPageId=" + fullId + ']');
+                        "restore using existing WAL) [fullPageId=" + fullId + ']', e);
 
                     buf.rewind();
 
@@ -1252,8 +1252,8 @@ public class PageMemoryImpl implements PageMemoryEx {
 
             GridUnsafe.copyMemory(absPtr + PAGE_OVERHEAD, tmpPtr, pageSize());
 
-            assert GridUnsafe.getInt(absPtr + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480
-            assert GridUnsafe.getInt(tmpPtr + 4) == 0; //TODO GG-11480
+            assert PageIO.getCrc(absPtr + PAGE_OVERHEAD) == 0; //TODO GG-11480
+            assert PageIO.getCrc(tmpPtr) == 0; //TODO GG-11480
         }
         else {
             byte[] arr = buf.array();
@@ -1402,7 +1402,7 @@ public class PageMemoryImpl implements PageMemoryEx {
         if (touch)
             PageHeader.writeTimestamp(absPtr, U.currentTimeMillis());
 
-        assert GridUnsafe.getInt(absPtr + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480
+        assert PageIO.getCrc(absPtr + PAGE_OVERHEAD) == 0; //TODO GG-11480
 
         return absPtr + PAGE_OVERHEAD;
     }
@@ -1490,11 +1490,11 @@ public class PageMemoryImpl implements PageMemoryEx {
             PageHeader.dirty(absPtr, false);
             PageHeader.tempBufferPointer(absPtr, tmpRelPtr);
 
-            assert GridUnsafe.getInt(absPtr + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480
-            assert GridUnsafe.getInt(tmpAbsPtr + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480
+            assert PageIO.getCrc(absPtr + PAGE_OVERHEAD) == 0; //TODO GG-11480
+            assert PageIO.getCrc(tmpAbsPtr + PAGE_OVERHEAD) == 0; //TODO GG-11480
         }
 
-        assert GridUnsafe.getInt(absPtr + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480
+        assert PageIO.getCrc(absPtr + PAGE_OVERHEAD) == 0; //TODO GG-11480
 
         return absPtr + PAGE_OVERHEAD;
     }
@@ -1522,7 +1522,7 @@ public class PageMemoryImpl implements PageMemoryEx {
 
         boolean pageWalRec = markDirty && walPlc != FALSE && (walPlc == TRUE || !wasDirty);
 
-        assert GridUnsafe.getInt(page + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480
+        assert PageIO.getCrc(page + PAGE_OVERHEAD) == 0; //TODO GG-11480
 
         if (markDirty)
             setDirty(fullId, page, markDirty, false);


[17/50] [abbrv] ignite git commit: IGNITE-10330: Disk page compression. - Fixes #5200.

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/compress/licenses/lz4-LICENSE.txt
----------------------------------------------------------------------
diff --git a/modules/compress/licenses/lz4-LICENSE.txt b/modules/compress/licenses/lz4-LICENSE.txt
new file mode 100644
index 0000000..c221aeb
--- /dev/null
+++ b/modules/compress/licenses/lz4-LICENSE.txt
@@ -0,0 +1,11 @@
+This repository uses 2 different licenses :
+- all files in the `lib` directory use a BSD 2-Clause license
+- all other files use a GPLv2 license, unless explicitly stated otherwise
+
+Relevant license is reminded at the top of each source file,
+and with presence of COPYING or LICENSE file in associated directories.
+
+This model is selected to emphasize that
+files in the `lib` directory are designed to be included into 3rd party applications,
+while all other files, in `programs`, `tests` or `examples`,
+receive more limited attention and support for such scenario.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/compress/licenses/lz4-java-LICENSE.txt
----------------------------------------------------------------------
diff --git a/modules/compress/licenses/lz4-java-LICENSE.txt b/modules/compress/licenses/lz4-java-LICENSE.txt
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/modules/compress/licenses/lz4-java-LICENSE.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/ignite/blob/e8eeea37/modules/compress/pom.xml
----------------------------------------------------------------------
diff --git a/modules/compress/pom.xml b/modules/compress/pom.xml
new file mode 100644
index 0000000..876a121
--- /dev/null
+++ b/modules/compress/pom.xml
@@ -0,0 +1,113 @@
+<?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>
+
+    <parent>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite-parent</artifactId>
+        <version>1</version>
+        <relativePath>../../parent</relativePath>
+    </parent>
+
+    <artifactId>ignite-compress</artifactId>
+    <version>2.7.0-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.github.jnr</groupId>
+            <artifactId>jnr-posix</artifactId>
+            <version>${jnr.posix.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.github.luben</groupId>
+            <artifactId>zstd-jni</artifactId>
+            <version>${zstd.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.lz4</groupId>
+            <artifactId>lz4-java</artifactId>
+            <version>${lz4.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.xerial.snappy</groupId>
+            <artifactId>snappy-java</artifactId>
+            <version>${snappy.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-context</artifactId>
+            <version>${spring.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>com.thoughtworks.xstream</groupId>
+            <artifactId>xstream</artifactId>
+            <version>1.4.8</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/CompressionProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/CompressionProcessorImpl.java b/modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/CompressionProcessorImpl.java
new file mode 100644
index 0000000..2553371
--- /dev/null
+++ b/modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/CompressionProcessorImpl.java
@@ -0,0 +1,403 @@
+/*
+ * 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.internal.processors.compress;
+
+import com.github.luben.zstd.Zstd;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import net.jpountz.lz4.LZ4Compressor;
+import net.jpountz.lz4.LZ4Factory;
+import net.jpountz.lz4.LZ4FastDecompressor;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.configuration.DiskPageCompression;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.CompactablePageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.xerial.snappy.Snappy;
+
+import static org.apache.ignite.configuration.DataStorageConfiguration.MAX_PAGE_SIZE;
+import static org.apache.ignite.configuration.DiskPageCompression.SKIP_GARBAGE;
+import static org.apache.ignite.internal.util.GridUnsafe.NATIVE_BYTE_ORDER;
+
+/**
+ * Compression processor.
+ */
+public class CompressionProcessorImpl extends CompressionProcessor {
+    /** Max page size. */
+    private final ThreadLocalByteBuffer compactBuf = new ThreadLocalByteBuffer(MAX_PAGE_SIZE);
+
+    /** A bit more than max page size. */
+    private final ThreadLocalByteBuffer compressBuf = new ThreadLocalByteBuffer(MAX_PAGE_SIZE + 1024);
+
+    /**
+     * @param ctx Kernal context.
+     */
+    @SuppressWarnings("WeakerAccess")
+    public CompressionProcessorImpl(GridKernalContext ctx) {
+        super(ctx);
+    }
+
+    /**
+     * @param cap Capacity.
+     * @return Direct byte buffer.
+     */
+    static ByteBuffer allocateDirectBuffer(int cap) {
+        return ByteBuffer.allocateDirect(cap).order(NATIVE_BYTE_ORDER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void checkPageCompressionSupported(Path storagePath, int pageSize) throws IgniteCheckedException {
+        if (!U.isLinux())
+            throw new IgniteCheckedException("Currently page compression is supported only for Linux.");
+
+        FileSystemUtils.checkSupported();
+
+        int fsBlockSize = FileSystemUtils.getFileSystemBlockSize(storagePath);
+
+        if (fsBlockSize <= 0)
+            throw new IgniteCheckedException("Failed to get file system block size: " + storagePath);
+
+        if (!U.isPow2(fsBlockSize))
+            throw new IgniteCheckedException("Storage block size must be power of 2: " + fsBlockSize);
+
+        if (pageSize < fsBlockSize * 2) {
+            throw new IgniteCheckedException("Page size (now configured to " + pageSize + " bytes) " +
+                "must be at least 2 times larger than the underlying storage block size (detected to be " + fsBlockSize +
+                " bytes at '" + storagePath + "') for page compression.");
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public ByteBuffer compressPage(
+        ByteBuffer page,
+        int pageSize,
+        int blockSize,
+        DiskPageCompression compression,
+        int compressLevel
+    ) throws IgniteCheckedException {
+        assert compression != null;
+        assert U.isPow2(pageSize): pageSize;
+        assert U.isPow2(blockSize): blockSize;
+        assert page.position() == 0 && page.limit() == pageSize;
+
+        PageIO io = PageIO.getPageIO(page);
+
+        if (!(io instanceof CompactablePageIO))
+            return page;
+
+        ByteBuffer compactPage = compactBuf.get();
+
+        // Drop the garbage from the page.
+        ((CompactablePageIO)io).compactPage(page, compactPage, pageSize);
+        page.clear();
+
+        int compactSize = compactPage.limit();
+
+        assert compactSize <= pageSize: compactSize;
+
+        // If no need to compress further or configured just to skip garbage.
+        if (compactSize < blockSize || compression == SKIP_GARBAGE)
+            return setCompactionInfo(compactPage, compactSize);
+
+        ByteBuffer compressedPage = doCompressPage(compression, compactPage, compactSize, compressLevel);
+
+        assert compressedPage.position() == 0;
+        int compressedSize = compressedPage.limit();
+
+        int freeCompactBlocks = (pageSize - compactSize) / blockSize;
+        int freeCompressedBlocks = (pageSize - compressedSize) / blockSize;
+
+        if (freeCompactBlocks >= freeCompressedBlocks) {
+            if (freeCompactBlocks == 0)
+                return page; // No blocks will be released.
+
+            return setCompactionInfo(compactPage, compactSize);
+        }
+
+        return setCompressionInfo(compressedPage, compression, compressedSize, compactSize);
+    }
+
+    /**
+     * @param page Page.
+     * @param compactSize Compacted page size.
+     * @return The given page.
+     */
+    private static ByteBuffer setCompactionInfo(ByteBuffer page, int compactSize) {
+        return setCompressionInfo(page, SKIP_GARBAGE, compactSize, compactSize);
+    }
+
+    /**
+     * @param page Page.
+     * @param compression Compression algorithm.
+     * @param compressedSize Compressed size.
+     * @param compactedSize Compact size.
+     * @return The given page.
+     */
+    private static ByteBuffer setCompressionInfo(ByteBuffer page, DiskPageCompression compression, int compressedSize, int compactedSize) {
+        assert compressedSize >= 0 && compressedSize <= Short.MAX_VALUE: compressedSize;
+        assert compactedSize >= 0 && compactedSize <= Short.MAX_VALUE: compactedSize;
+
+        PageIO.setCompressionType(page, getCompressionType(compression));
+        PageIO.setCompressedSize(page, (short)compressedSize);
+        PageIO.setCompactedSize(page, (short)compactedSize);
+
+        return page;
+    }
+
+    /**
+     * @param compression Compression algorithm.
+     * @param compactPage Compacted page.
+     * @param compactSize Compacted page size.
+     * @param compressLevel Compression level.
+     * @return Compressed page.
+     */
+    private ByteBuffer doCompressPage(DiskPageCompression compression, ByteBuffer compactPage, int compactSize, int compressLevel) {
+        switch (compression) {
+            case ZSTD:
+                return compressPageZstd(compactPage, compactSize, compressLevel);
+
+            case LZ4:
+                return compressPageLz4(compactPage, compactSize, compressLevel);
+
+            case SNAPPY:
+                return compressPageSnappy(compactPage, compactSize);
+        }
+        throw new IllegalStateException("Unsupported compression: " + compression);
+    }
+
+    /**
+     * @param compactPage Compacted page.
+     * @param compactSize Compacted page size.
+     * @param compressLevel Compression level.
+     * @return Compressed page.
+     */
+    private ByteBuffer compressPageLz4(ByteBuffer compactPage, int compactSize, int compressLevel) {
+        LZ4Compressor compressor = Lz4.getCompressor(compressLevel);
+
+        ByteBuffer compressedPage = compressBuf.get();
+
+        copyPageHeader(compactPage, compressedPage, compactSize);
+        compressor.compress(compactPage, compressedPage);
+
+        compactPage.flip();
+        compressedPage.flip();
+
+        return compressedPage;
+    }
+
+    /**
+     * @param compactPage Compacted page.
+     * @param compactSize Compacted page size.
+     * @param compressLevel Compression level.
+     * @return Compressed page.
+     */
+    private ByteBuffer compressPageZstd(ByteBuffer compactPage, int compactSize, int compressLevel) {
+        ByteBuffer compressedPage = compressBuf.get();
+
+        copyPageHeader(compactPage, compressedPage, compactSize);
+        Zstd.compress(compressedPage, compactPage, compressLevel);
+
+        compactPage.flip();
+        compressedPage.flip();
+
+        return compressedPage;
+    }
+
+    /**
+     * @param compactPage Compacted page.
+     * @param compactSize Compacted page size.
+     * @return Compressed page.
+     */
+    private ByteBuffer compressPageSnappy(ByteBuffer compactPage, int compactSize) {
+        ByteBuffer compressedPage = compressBuf.get();
+
+        copyPageHeader(compactPage, compressedPage, compactSize);
+
+        try {
+            int compressedSize = Snappy.compress(compactPage, compressedPage);
+            assert compressedPage.limit() == PageIO.COMMON_HEADER_END + compressedSize;
+        }
+        catch (IOException e) {
+            throw new IgniteException("Failed to compress page with Snappy.", e);
+        }
+
+        compactPage.position(0);
+        compressedPage.position(0);
+
+        return compressedPage;
+    }
+
+    /**
+     * @param compactPage Compacted page.
+     * @param compressedPage Compressed page.
+     * @param compactSize Compacted page size.
+     */
+    private static void copyPageHeader(ByteBuffer compactPage, ByteBuffer compressedPage, int compactSize) {
+        compactPage.limit(PageIO.COMMON_HEADER_END);
+        compressedPage.put(compactPage);
+        compactPage.limit(compactSize);
+    }
+
+    /**
+     * @param compression Compression.
+     * @return Level.
+     */
+    private static byte getCompressionType(DiskPageCompression compression) {
+        if (compression == null)
+            return UNCOMPRESSED_PAGE;
+
+        switch (compression) {
+            case ZSTD:
+                return ZSTD_COMPRESSED_PAGE;
+
+            case LZ4:
+                return LZ4_COMPRESSED_PAGE;
+
+            case SNAPPY:
+                return SNAPPY_COMPRESSED_PAGE;
+
+            case SKIP_GARBAGE:
+                return COMPACTED_PAGE;
+        }
+        throw new IllegalStateException("Unexpected compression: " + compression);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void decompressPage(ByteBuffer page, int pageSize) throws IgniteCheckedException {
+        assert page.capacity() == pageSize;
+
+        byte compressType = PageIO.getCompressionType(page);
+
+        if (compressType == UNCOMPRESSED_PAGE)
+            return; // Nothing to do.
+
+        short compressedSize = PageIO.getCompressedSize(page);
+        short compactSize = PageIO.getCompactedSize(page);
+
+        assert compactSize <= pageSize && compactSize >= compressedSize;
+
+        if (compressType == COMPACTED_PAGE) {
+            // Just setup bounds before restoring the page.
+            page.position(0).limit(compactSize);
+        }
+        else {
+            ByteBuffer dst = compressBuf.get();
+
+            // Position on a part that needs to be decompressed.
+            page.limit(compressedSize)
+                .position(PageIO.COMMON_HEADER_END);
+
+            // LZ4 needs this limit to be exact.
+            dst.limit(compactSize - PageIO.COMMON_HEADER_END);
+
+            switch (compressType) {
+                case ZSTD_COMPRESSED_PAGE:
+                    Zstd.decompress(dst, page);
+                    dst.flip();
+
+                    break;
+
+                case LZ4_COMPRESSED_PAGE:
+                    Lz4.decompress(page, dst);
+                    dst.flip();
+
+                    break;
+
+                case SNAPPY_COMPRESSED_PAGE:
+                    try {
+                        Snappy.uncompress(page, dst);
+                    }
+                    catch (IOException e) {
+                        throw new IgniteException(e);
+                    }
+                    break;
+
+                default:
+                    throw new IgniteException("Unknown compression: " + compressType);
+            }
+
+            page.position(PageIO.COMMON_HEADER_END).limit(compactSize);
+            page.put(dst).flip();
+            assert page.limit() == compactSize;
+        }
+
+        CompactablePageIO io = PageIO.getPageIO(page);
+
+        io.restorePage(page, pageSize);
+
+        setCompressionInfo(page, null, 0, 0);
+    }
+
+    /** */
+    static class Lz4 {
+        /** */
+        static final LZ4Factory factory = LZ4Factory.fastestInstance();
+
+        /** */
+        static final LZ4FastDecompressor decompressor = factory.fastDecompressor();
+
+        /** */
+        static final LZ4Compressor fastCompressor = factory.fastCompressor();
+
+        /**
+         * @param level Compression level.
+         * @return Compressor.
+         */
+        static LZ4Compressor getCompressor(int level) {
+            assert level >= 0 && level <= 17: level;
+            return level == 0 ? fastCompressor : factory.highCompressor(level);
+        }
+
+        /**
+         * @param page Page.
+         * @param dst Destination buffer.
+         */
+        static void decompress(ByteBuffer page, ByteBuffer dst) {
+            decompressor.decompress(page, dst);
+        }
+    }
+
+    /**
+     */
+    static final class ThreadLocalByteBuffer extends ThreadLocal<ByteBuffer> {
+        /** */
+        final int size;
+
+        /**
+         * @param size Size.
+         */
+        ThreadLocalByteBuffer(int size) {
+            this.size = size;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected ByteBuffer initialValue() {
+            return allocateDirectBuffer(size);
+        }
+
+        /** {@inheritDoc} */
+        @Override public ByteBuffer get() {
+            ByteBuffer buf = super.get();
+            buf.clear();
+            return buf;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/NativeFileSystemLinux.java
----------------------------------------------------------------------
diff --git a/modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/NativeFileSystemLinux.java b/modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/NativeFileSystemLinux.java
new file mode 100644
index 0000000..22d4926
--- /dev/null
+++ b/modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/NativeFileSystemLinux.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.internal.processors.compress;
+
+import jnr.ffi.LibraryLoader;
+import org.apache.ignite.IgniteException;
+
+/**
+ * Linux native file system API.
+ */
+public final class NativeFileSystemLinux extends NativeFileSystemPosix {
+    /**
+     * default is extend size
+     */
+    public static final int FALLOC_FL_KEEP_SIZE = 0x01;
+
+    /**
+     * de-allocates range
+     */
+    public static final int FALLOC_FL_PUNCH_HOLE = 0x02;
+
+    /**
+     * reserved codepoint
+     */
+    public static final int FALLOC_FL_NO_HIDE_STALE = 0x04;
+
+    /**
+     * FALLOC_FL_COLLAPSE_RANGE is used to remove a range of a file
+     * without leaving a hole in the file. The contents of the file beyond
+     * the range being removed is appended to the start offset of the range
+     * being removed (i.e. the hole that was punched is "collapsed"),
+     * resulting in a file layout that looks like the range that was
+     * removed never existed. As such collapsing a range of a file changes
+     * the size of the file, reducing it by the same length of the range
+     * that has been removed by the operation.
+     *
+     * Different filesystems may implement different limitations on the
+     * granularity of the operation. Most will limit operations to
+     * filesystem block size boundaries, but this boundary may be larger or
+     * smaller depending on the filesystem and/or the configuration of the
+     * filesystem or file.
+     *
+     * Attempting to collapse a range that crosses the end of the file is
+     * considered an illegal operation - just use ftruncate(2) if you need
+     * to collapse a range that crosses EOF.
+     */
+    public static final int FALLOC_FL_COLLAPSE_RANGE = 0x08;
+
+    /**
+     * FALLOC_FL_ZERO_RANGE is used to convert a range of file to zeros preferably
+     * without issuing data IO. Blocks should be preallocated for the regions that
+     * span holes in the file, and the entire range is preferable converted to
+     * unwritten extents - even though file system may choose to zero out the
+     * extent or do whatever which will result in reading zeros from the range
+     * while the range remains allocated for the file.
+     *
+     * This can be also used to preallocate blocks past EOF in the same way as
+     * with fallocate. Flag FALLOC_FL_KEEP_SIZE should cause the inode
+     * size to remain the same.
+     */
+    public static final int FALLOC_FL_ZERO_RANGE = 0x10;
+
+    /**
+     * FALLOC_FL_INSERT_RANGE is use to insert space within the file size without
+     * overwriting any existing data. The contents of the file beyond offset are
+     * shifted towards right by len bytes to create a hole.  As such, this
+     * operation will increase the size of the file by len bytes.
+     *
+     * Different filesystems may implement different limitations on the granularity
+     * of the operation. Most will limit operations to filesystem block size
+     * boundaries, but this boundary may be larger or smaller depending on
+     * the filesystem and/or the configuration of the filesystem or file.
+     *
+     * Attempting to insert space using this flag at OR beyond the end of
+     * the file is considered an illegal operation - just use ftruncate(2) or
+     * fallocate(2) with mode 0 for such type of operations.
+     */
+    public static final int FALLOC_FL_INSERT_RANGE = 0x20;
+
+    /**
+     * FALLOC_FL_UNSHARE_RANGE is used to unshare shared blocks within the
+     * file size without overwriting any existing data. The purpose of this
+     * call is to preemptively reallocate any blocks that are subject to
+     * copy-on-write.
+     *
+     * Different filesystems may implement different limitations on the
+     * granularity of the operation. Most will limit operations to filesystem
+     * block size boundaries, but this boundary may be larger or smaller
+     * depending on the filesystem and/or the configuration of the filesystem
+     * or file.
+     *
+     * This flag can only be used with allocate-mode fallocate, which is
+     * to say that it cannot be used with the punch, zero, collapse, or
+     * insert range modes.
+     */
+    public static final int FALLOC_FL_UNSHARE_RANGE = 0x40;
+
+    /** */
+    private static final LinuxNativeLibC libc = LibraryLoader.create(LinuxNativeLibC.class)
+        .failImmediately().load("c");
+
+    /** {@inheritDoc} */
+    @Override public void punchHole(int fd, long off, long len) {
+        int res = libc.fallocate(fd, FALLOC_FL_PUNCH_HOLE | FALLOC_FL_KEEP_SIZE, off, len);
+
+        if (res != 0)
+            throw new IgniteException("errno: " + res);
+    }
+
+    /**
+     */
+    public interface LinuxNativeLibC {
+        /**
+         * Allows the caller to directly manipulate the allocated
+         * disk space for the file referred to by fd for the byte range starting
+         * at {@code off} offset and continuing for {@code len} bytes.
+         *
+         * @param fd   file descriptor.
+         * @param mode determines the operation to be performed on the given range.
+         * @param off  required position offset.
+         * @param len  required length.
+         * @return On success, fallocate() returns zero.  On error, -1 is returned and
+         * {@code errno} is set to indicate the error.
+         */
+        int fallocate(int fd, int mode, long off, long len);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/NativeFileSystemPosix.java
----------------------------------------------------------------------
diff --git a/modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/NativeFileSystemPosix.java b/modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/NativeFileSystemPosix.java
new file mode 100644
index 0000000..fcf485f
--- /dev/null
+++ b/modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/NativeFileSystemPosix.java
@@ -0,0 +1,54 @@
+/*
+ * 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.internal.processors.compress;
+
+import java.nio.file.Path;
+import jnr.posix.FileStat;
+import jnr.posix.POSIX;
+import jnr.posix.POSIXFactory;
+
+/**
+ * Posix file system API.
+ */
+public class NativeFileSystemPosix implements NativeFileSystem {
+    /** */
+    private static POSIX posix = POSIXFactory.getPOSIX();
+
+    /** {@inheritDoc} */
+    @Override public int getFileSystemBlockSize(Path path) {
+        FileStat stat = posix.stat(path.toString());
+        return Math.toIntExact(stat.blockSize());
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getFileSystemBlockSize(int fd) {
+        FileStat stat = posix.fstat(fd);
+        return Math.toIntExact(stat.blockSize());
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSparseFileSize(int fd) {
+        FileStat stat = posix.fstat(fd);
+        return stat.blocks() * 512;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void punchHole(int fd, long off, long len) {
+        throw new UnsupportedOperationException();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/compress/src/test/java/org/apache/ignite/internal/processors/compress/CompressionProcessorTest.java
----------------------------------------------------------------------
diff --git a/modules/compress/src/test/java/org/apache/ignite/internal/processors/compress/CompressionProcessorTest.java b/modules/compress/src/test/java/org/apache/ignite/internal/processors/compress/CompressionProcessorTest.java
new file mode 100644
index 0000000..f660426
--- /dev/null
+++ b/modules/compress/src/test/java/org/apache/ignite/internal/processors/compress/CompressionProcessorTest.java
@@ -0,0 +1,1021 @@
+/*
+ * 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.internal.processors.compress;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.Function;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.configuration.DiskPageCompression;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.PageUtils;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPagePayload;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.SimpleDataPageIO;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.testframework.junits.GridTestKernalContext;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.configuration.DiskPageCompression.LZ4;
+import static org.apache.ignite.configuration.DiskPageCompression.SKIP_GARBAGE;
+import static org.apache.ignite.configuration.DiskPageCompression.SNAPPY;
+import static org.apache.ignite.configuration.DiskPageCompression.ZSTD;
+import static org.apache.ignite.internal.processors.compress.CompressionProcessor.LZ4_MAX_LEVEL;
+import static org.apache.ignite.internal.processors.compress.CompressionProcessor.LZ4_MIN_LEVEL;
+import static org.apache.ignite.internal.processors.compress.CompressionProcessor.UNCOMPRESSED_PAGE;
+import static org.apache.ignite.internal.processors.compress.CompressionProcessor.ZSTD_MAX_LEVEL;
+import static org.apache.ignite.internal.processors.compress.CompressionProcessorImpl.allocateDirectBuffer;
+import static org.apache.ignite.internal.processors.compress.CompressionProcessorTest.TestInnerIO.INNER_IO;
+import static org.apache.ignite.internal.processors.compress.CompressionProcessorTest.TestLeafIO.LEAF_IO;
+import static org.apache.ignite.internal.util.GridUnsafe.bufferAddress;
+
+/**
+ */
+public class CompressionProcessorTest extends GridCommonAbstractTest {
+    /** */
+    private static final int ITEM_SIZE = 6; // To fill the whole page.
+
+    /** */
+    private int blockSize = 16;
+
+    /** */
+    private int pageSize = 4 * 1024;
+
+    /** */
+    private DiskPageCompression compression;
+
+    /** */
+    private int compressLevel;
+
+    /** */
+    private CompressionProcessor p;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        PageIO.registerTest(INNER_IO, LEAF_IO);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() {
+        p = new CompressionProcessorImpl(new GridTestKernalContext(log));
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testDataPageCompact16() throws IgniteCheckedException {
+        blockSize = 16;
+        compression = SKIP_GARBAGE;
+
+        doTestDataPage();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testDataPageCompact128() throws IgniteCheckedException {
+        blockSize = 128;
+        compression = SKIP_GARBAGE;
+
+        doTestDataPage();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testDataPageCompact1k() throws IgniteCheckedException {
+        blockSize = 1024;
+        compression = SKIP_GARBAGE;
+
+        doTestDataPage();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testDataPageCompact2k() throws IgniteCheckedException {
+        blockSize = 2 * 1024;
+        compression = SKIP_GARBAGE;
+
+        doTestDataPage();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testDataPageZstd16() throws IgniteCheckedException {
+        blockSize = 16;
+        compression = ZSTD;
+        compressLevel = ZSTD_MAX_LEVEL;
+
+        doTestDataPage();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testDataPageZstd128() throws IgniteCheckedException {
+        blockSize = 128;
+        compression = ZSTD;
+        compressLevel = ZSTD_MAX_LEVEL;
+
+        doTestDataPage();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testDataPageZstd1k() throws IgniteCheckedException {
+        blockSize = 1024;
+        compression = ZSTD;
+        compressLevel = ZSTD_MAX_LEVEL;
+
+        doTestDataPage();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testDataPageZstd2k() throws IgniteCheckedException {
+        blockSize = 2 * 1024;
+        compression = ZSTD;
+        compressLevel = ZSTD_MAX_LEVEL;
+
+        doTestDataPage();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testDataPageSnappy16() throws IgniteCheckedException {
+        blockSize = 16;
+        compression = SNAPPY;
+
+        doTestDataPage();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testDataPageSnappy128() throws IgniteCheckedException {
+        blockSize = 128;
+        compression = SNAPPY;
+
+        doTestDataPage();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testDataPageSnappy1k() throws IgniteCheckedException {
+        blockSize = 1024;
+        compression = SNAPPY;
+
+        doTestDataPage();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testDataPageSnappy2k() throws IgniteCheckedException {
+        blockSize = 2 * 1024;
+        compression = SNAPPY;
+
+        doTestDataPage();
+    }
+
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testDataPageLz4Fast16() throws IgniteCheckedException {
+        blockSize = 16;
+        compression = LZ4;
+        compressLevel = LZ4_MIN_LEVEL;
+
+        doTestDataPage();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testDataPageLz4Fast128() throws IgniteCheckedException {
+        blockSize = 128;
+        compression = LZ4;
+        compressLevel = LZ4_MIN_LEVEL;
+
+        doTestDataPage();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testDataPageLz4Fast1k() throws IgniteCheckedException {
+        blockSize = 1024;
+        compression = LZ4;
+        compressLevel = LZ4_MIN_LEVEL;
+
+        doTestDataPage();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testDataPageLz4Fast2k() throws IgniteCheckedException {
+        blockSize = 2 * 1024;
+        compression = LZ4;
+        compressLevel = LZ4_MIN_LEVEL;
+
+        doTestDataPage();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testDataPageLz4Slow16() throws IgniteCheckedException {
+        blockSize = 16;
+        compression = LZ4;
+        compressLevel = LZ4_MAX_LEVEL;
+
+        doTestDataPage();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testDataPageLz4Slow128() throws IgniteCheckedException {
+        blockSize = 128;
+        compression = LZ4;
+        compressLevel = LZ4_MAX_LEVEL;
+
+        doTestDataPage();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testDataPageLz4Slow1k() throws IgniteCheckedException {
+        blockSize = 1024;
+        compression = LZ4;
+        compressLevel = LZ4_MAX_LEVEL;
+
+        doTestDataPage();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testDataPageLz4Slow2k() throws IgniteCheckedException {
+        blockSize = 2 * 1024;
+        compression = LZ4;
+        compressLevel = LZ4_MAX_LEVEL;
+
+        doTestDataPage();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInnerPageCompact16() throws IgniteCheckedException {
+        blockSize = 16;
+        compression = SKIP_GARBAGE;
+
+        doTestBTreePage(INNER_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testLeafPageCompact16() throws IgniteCheckedException {
+        blockSize = 16;
+        compression = SKIP_GARBAGE;
+
+        doTestBTreePage(LEAF_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInnerPageZstd16() throws IgniteCheckedException {
+        blockSize = 16;
+        compression = ZSTD;
+        compressLevel = ZSTD_MAX_LEVEL;
+
+        doTestBTreePage(INNER_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testLeafPageZstd16() throws IgniteCheckedException {
+        blockSize = 16;
+        compression = ZSTD;
+        compressLevel = ZSTD_MAX_LEVEL;
+
+        doTestBTreePage(LEAF_IO);
+    }
+
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInnerPageLz4Fast16() throws IgniteCheckedException {
+        blockSize = 16;
+        compression = LZ4;
+        compressLevel = LZ4_MIN_LEVEL;
+
+        doTestBTreePage(INNER_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testLeafPageLz4Fast16() throws IgniteCheckedException {
+        blockSize = 16;
+        compression = LZ4;
+        compressLevel = LZ4_MIN_LEVEL;
+
+        doTestBTreePage(LEAF_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInnerPageLz4Slow16() throws IgniteCheckedException {
+        blockSize = 16;
+        compression = LZ4;
+        compressLevel = LZ4_MAX_LEVEL;
+
+        doTestBTreePage(INNER_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testLeafPageLz4Slow16() throws IgniteCheckedException {
+        blockSize = 16;
+        compression = LZ4;
+        compressLevel = LZ4_MAX_LEVEL;
+
+        doTestBTreePage(LEAF_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInnerPageSnappy16() throws IgniteCheckedException {
+        blockSize = 16;
+        compression = SNAPPY;
+
+        doTestBTreePage(INNER_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testLeafPageSnappy16() throws IgniteCheckedException {
+        blockSize = 16;
+        compression = SNAPPY;
+
+        doTestBTreePage(LEAF_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInnerPageCompact128() throws IgniteCheckedException {
+        blockSize = 128;
+        compression = SKIP_GARBAGE;
+
+        doTestBTreePage(INNER_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testLeafPageCompact128() throws IgniteCheckedException {
+        blockSize = 128;
+        compression = SKIP_GARBAGE;
+
+        doTestBTreePage(LEAF_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInnerPageZstd128() throws IgniteCheckedException {
+        blockSize = 128;
+        compression = ZSTD;
+        compressLevel = ZSTD_MAX_LEVEL;
+
+        doTestBTreePage(INNER_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testLeafPageZstd128() throws IgniteCheckedException {
+        blockSize = 128;
+        compression = ZSTD;
+        compressLevel = ZSTD_MAX_LEVEL;
+
+        doTestBTreePage(LEAF_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInnerPageLz4Fast128() throws IgniteCheckedException {
+        blockSize = 128;
+        compression = LZ4;
+        compressLevel = LZ4_MIN_LEVEL;
+
+        doTestBTreePage(INNER_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testLeafPageLz4Fast128() throws IgniteCheckedException {
+        blockSize = 128;
+        compression = LZ4;
+        compressLevel = LZ4_MIN_LEVEL;
+
+        doTestBTreePage(LEAF_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInnerPageLz4Slow128() throws IgniteCheckedException {
+        blockSize = 128;
+        compression = LZ4;
+        compressLevel = LZ4_MAX_LEVEL;
+
+        doTestBTreePage(INNER_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testLeafPageLz4Slow128() throws IgniteCheckedException {
+        blockSize = 128;
+        compression = LZ4;
+        compressLevel = LZ4_MAX_LEVEL;
+
+        doTestBTreePage(LEAF_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInnerPageSnappy128() throws IgniteCheckedException {
+        blockSize = 128;
+        compression = SNAPPY;
+
+        doTestBTreePage(INNER_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testLeafPageSnappy128() throws IgniteCheckedException {
+        blockSize = 128;
+        compression = SNAPPY;
+
+        doTestBTreePage(LEAF_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInnerPageCompact1k() throws IgniteCheckedException {
+        blockSize = 1024;
+        compression = SKIP_GARBAGE;
+
+        doTestBTreePage(INNER_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testLeafPageCompact1k() throws IgniteCheckedException {
+        blockSize = 1024;
+        compression = SKIP_GARBAGE;
+
+        doTestBTreePage(LEAF_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInnerPageZstd1k() throws IgniteCheckedException {
+        blockSize = 1024;
+        compression = ZSTD;
+        compressLevel = ZSTD_MAX_LEVEL;
+
+        doTestBTreePage(INNER_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testLeafPageZstd1k() throws IgniteCheckedException {
+        blockSize = 1024;
+        compression = ZSTD;
+        compressLevel = ZSTD_MAX_LEVEL;
+
+        doTestBTreePage(LEAF_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInnerPageLz4Fast1k() throws IgniteCheckedException {
+        blockSize = 1024;
+        compression = LZ4;
+        compressLevel = LZ4_MIN_LEVEL;
+
+        doTestBTreePage(INNER_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testLeafPageLz4Fast1k() throws IgniteCheckedException {
+        blockSize = 1024;
+        compression = LZ4;
+        compressLevel = LZ4_MIN_LEVEL;
+
+        doTestBTreePage(LEAF_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInnerPageLz4Slow1k() throws IgniteCheckedException {
+        blockSize = 1024;
+        compression = LZ4;
+        compressLevel = LZ4_MAX_LEVEL;
+
+        doTestBTreePage(INNER_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testLeafPageLz4Slow1k() throws IgniteCheckedException {
+        blockSize = 1024;
+        compression = LZ4;
+        compressLevel = LZ4_MAX_LEVEL;
+
+        doTestBTreePage(LEAF_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInnerPageSnappy1k() throws IgniteCheckedException {
+        blockSize = 1024;
+        compression = SNAPPY;
+
+        doTestBTreePage(INNER_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testLeafPageSnappy1k() throws IgniteCheckedException {
+        blockSize = 1024;
+        compression = SNAPPY;
+
+        doTestBTreePage(LEAF_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInnerPageCompact2k() throws IgniteCheckedException {
+        blockSize = 2 * 1024;
+        compression = SKIP_GARBAGE;
+
+        doTestBTreePage(INNER_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testLeafPageCompact2k() throws IgniteCheckedException {
+        blockSize = 2 * 1024;
+        compression = SKIP_GARBAGE;
+
+        doTestBTreePage(LEAF_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInnerPageZstd2k() throws IgniteCheckedException {
+        blockSize = 2 * 1024;
+        compression = ZSTD;
+        compressLevel = ZSTD_MAX_LEVEL;
+
+        doTestBTreePage(INNER_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testLeafPageZstd2k() throws IgniteCheckedException {
+        blockSize = 2 * 1024;
+        compression = ZSTD;
+        compressLevel = ZSTD_MAX_LEVEL;
+
+        doTestBTreePage(LEAF_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInnerPageLz4Fast2k() throws IgniteCheckedException {
+        blockSize = 2 * 1024;
+        compression = LZ4;
+        compressLevel = LZ4_MIN_LEVEL;
+
+        doTestBTreePage(INNER_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testLeafPageLz4Fast2k() throws IgniteCheckedException {
+        blockSize = 2 * 1024;
+        compression = LZ4;
+        compressLevel = LZ4_MIN_LEVEL;
+
+        doTestBTreePage(LEAF_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInnerPageLz4Slow2k() throws IgniteCheckedException {
+        blockSize = 2 * 1024;
+        compression = LZ4;
+        compressLevel = LZ4_MAX_LEVEL;
+
+        doTestBTreePage(INNER_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testLeafPageLz4Slow2k() throws IgniteCheckedException {
+        blockSize = 2 * 1024;
+        compression = LZ4;
+        compressLevel = LZ4_MAX_LEVEL;
+
+        doTestBTreePage(LEAF_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInnerPageSnappy2k() throws IgniteCheckedException {
+        blockSize = 2 * 1024;
+        compression = SNAPPY;
+
+        doTestBTreePage(INNER_IO);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testLeafPageSnappy2k() throws IgniteCheckedException {
+        blockSize = 2 * 1024;
+        compression = SNAPPY;
+
+        doTestBTreePage(LEAF_IO);
+    }
+
+    /**
+     * @param io Page IO.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void doTestBTreePage(BPlusIO<byte[]> io) throws IgniteCheckedException {
+        Random rnd = ThreadLocalRandom.current();
+
+        final byte[][] rows = new byte[3][io.getItemSize()];
+
+        for (int i = 0; i < rows.length; i++)
+            rnd.nextBytes(rows[i]);
+
+        ByteBuffer page = allocateDirectBuffer(pageSize);
+        long pageAddr = bufferAddress(page);
+
+        long pageId = PageIdUtils.pageId(PageIdAllocator.INDEX_PARTITION, PageIdAllocator.FLAG_IDX, 171717);
+
+        io.initNewPage(pageAddr, pageId, pageSize);
+
+        checkIo(io, page);
+
+        Function<ByteBuffer, List<?>> getContents = (buf) -> {
+            long addr = bufferAddress(buf);
+
+            int cnt = io.getCount(addr);
+
+            List<Object> list = new ArrayList<>(cnt);
+
+            for (int i = 0; i < cnt; i++) {
+                if (!io.isLeaf())
+                    list.add(((BPlusInnerIO)io).getLeft(addr, i));
+
+                try {
+                    list.add(new Bytes(io.getLookupRow(null, addr, i)));
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IllegalStateException(e);
+                }
+
+                if (!io.isLeaf())
+                    list.add(((BPlusInnerIO)io).getRight(addr, i));
+            }
+
+            return list;
+        };
+
+        // Empty page.
+        checkCompressDecompress(page, getContents, false);
+
+        int cnt = io.getMaxCount(pageAddr, pageSize);
+
+        for (int i = 0; i < cnt; i++) {
+            byte[] row = rows[rnd.nextInt(rows.length)];
+            io.insert(pageAddr, i, row, row, 777_000 + i, false);
+        }
+
+        if (io.isLeaf())
+            assertEquals(pageSize, io.getItemsEnd(pageAddr)); // Page must be full.
+
+        // Full page.
+        checkCompressDecompress(page, getContents, io.isLeaf());
+
+        io.setCount(pageAddr, cnt / 2);
+
+        // Half page.
+        checkCompressDecompress(page, getContents, false);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    private void doTestDataPage() throws IgniteCheckedException {
+        Random rnd = ThreadLocalRandom.current();
+
+        final byte[][] rows = new byte[][]{
+            new byte[17], new byte[37], new byte[71]
+        };
+
+        for (int i = 0; i < rows.length; i++)
+            rnd.nextBytes(rows[i]);
+
+        ByteBuffer page = allocateDirectBuffer(pageSize);
+        long pageAddr = bufferAddress(page);
+
+        SimpleDataPageIO io = SimpleDataPageIO.VERSIONS.latest();
+
+        long pageId = PageIdUtils.pageId(PageIdAllocator.MAX_PARTITION_ID, PageIdAllocator.FLAG_DATA, 171717);
+
+        io.initNewPage(pageAddr, pageId, pageSize);
+
+        checkIo(io, page);
+
+        Function<ByteBuffer,List<Bytes>> getContents = (buf) -> {
+            try {
+                long addr = bufferAddress(buf);
+
+                return io.forAllItems(addr, (link) -> {
+                    DataPagePayload payload = io.readPayload(addr, PageIdUtils.itemId(link), pageSize);
+
+                    return new Bytes(payload.getBytes(addr));
+                });
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException(e);
+            }
+        };
+
+        // Empty data page.
+        checkCompressDecompress(page, getContents, false);
+
+        GridIntList itemIds = new GridIntList();
+
+        for (;;) {
+            byte[] row = rows[rnd.nextInt(rows.length)];
+
+            if (io.getFreeSpace(pageAddr) < row.length)
+                break;
+
+            itemIds.add(io.addRow(pageAddr, row, pageSize));
+        }
+
+        int freeSpace = io.getFreeSpace(pageAddr);
+
+        if (freeSpace != 0) {
+            byte[] lastRow = new byte[freeSpace];
+            rnd.nextBytes(lastRow);
+
+            io.addRowFragment(pageId, pageAddr, lastRow, 777L, pageSize);
+
+            assertEquals(0, io.getRealFreeSpace(pageAddr));
+        }
+
+        // Full data page.
+        checkCompressDecompress(page, getContents, io.getRealFreeSpace(pageAddr) == 0);
+
+        for (int i = 0; i < itemIds.size(); i += 2)
+            io.removeRow(pageAddr, itemIds.get(i), pageSize);
+
+        // Half-filled data page.
+        checkCompressDecompress(page, getContents, false);
+    }
+
+    private void checkIo(PageIO io, ByteBuffer page) throws IgniteCheckedException {
+        assertSame(io, PageIO.getPageIO(bufferAddress(page)));
+        assertSame(io, PageIO.getPageIO(page));
+    }
+
+    private void checkCompressDecompress(ByteBuffer page, Function<ByteBuffer, ?> getPageContents, boolean fullPage)
+        throws IgniteCheckedException {
+        PageIO.setCrc(page, 0xABCDEF13);
+
+        long pageId = PageIO.getPageId(page);
+        PageIO io = PageIO.getPageIO(page);
+
+        ByteBuffer compressed = p.compressPage(page, pageSize, blockSize, compression, compressLevel);
+
+        int compressedSize = PageIO.getCompressedSize(compressed);
+
+        assertNotSame(page, compressed); // This is generally possible but not interesting in this test.
+
+        assertTrue(compressedSize > 0);
+        assertTrue(compressedSize <= pageSize);
+        assertEquals(compressedSize, compressed.limit());
+
+        if (!fullPage || compression != SKIP_GARBAGE)
+            assertTrue(pageSize > compressedSize);
+
+        assertEquals(0, compressed.position());
+
+        checkIo(io, compressed);
+        assertEquals(0, page.position());
+        assertEquals(pageSize, page.limit());
+
+        info(io.getClass().getSimpleName() + " " + compression + " " + compressLevel + ": " + compressedSize + "/" + pageSize);
+
+        if (!fullPage || compression != SKIP_GARBAGE)
+            assertTrue(compressedSize < pageSize);
+
+        assertEquals(pageId, PageIO.getPageId(compressed));
+
+        ByteBuffer decompress = allocateDirectBuffer(pageSize);
+        decompress.put(compressed).clear();
+
+        p.decompressPage(decompress, pageSize);
+
+        assertEquals(0, decompress.position());
+        assertEquals(pageSize, decompress.limit());
+
+        checkIo(io, decompress);
+        assertEquals(UNCOMPRESSED_PAGE, PageIO.getCompressionType(page));
+        assertEquals(0, PageIO.getCompressedSize(page));
+        assertEquals(0, PageIO.getCompactedSize(page));
+
+        assertTrue(Arrays.equals(getPageCommonHeader(page), getPageCommonHeader(decompress)));
+        assertEquals(getPageContents.apply(page), getPageContents.apply(decompress));
+    }
+
+    /**
+     * @param page Page.
+     * @return Page header.
+     */
+    private static byte[] getPageCommonHeader(ByteBuffer page) {
+        return PageUtils.getBytes(GridUnsafe.bufferAddress(page), 0, PageIO.COMMON_HEADER_END);
+    }
+
+    /**
+     */
+    private static class Bytes {
+        /** */
+        private final byte[] bytes;
+
+        /**
+         * @param bytes Bytes.
+         */
+        private Bytes(byte[] bytes) {
+            assert bytes != null;
+            this.bytes = bytes;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            Bytes bytes1 = (Bytes)o;
+
+            return Arrays.equals(bytes, bytes1.bytes);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return Arrays.hashCode(bytes);
+        }
+    }
+
+    /**
+     */
+    static class TestLeafIO extends BPlusLeafIO<byte[]> {
+        /** */
+        static final TestLeafIO LEAF_IO = new TestLeafIO();
+
+        /**
+         */
+        TestLeafIO() {
+            super(29_501, 1, ITEM_SIZE);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void storeByOffset(long pageAddr, int off, byte[] row) {
+            PageUtils.putBytes(pageAddr, off, row);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void store(long dstPageAddr, int dstIdx, BPlusIO<byte[]> srcIo, long srcPageAddr,
+            int srcIdx) throws IgniteCheckedException {
+            storeByOffset(dstPageAddr, offset(dstIdx), srcIo.getLookupRow(null, srcPageAddr, srcIdx));
+        }
+
+        /** {@inheritDoc} */
+        @Override public byte[] getLookupRow(BPlusTree<byte[],?> tree, long pageAddr, int idx) {
+            return PageUtils.getBytes(pageAddr, offset(idx), itemSize);
+        }
+    }
+
+    /**
+     */
+    static class TestInnerIO extends BPlusInnerIO<byte[]> {
+        /** */
+        static TestInnerIO INNER_IO = new TestInnerIO();
+
+        /**
+         */
+        TestInnerIO() {
+            super(29_502, 1, true, ITEM_SIZE);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void storeByOffset(long pageAddr, int off, byte[] row) {
+            PageUtils.putBytes(pageAddr, off, row);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void store(long dstPageAddr, int dstIdx, BPlusIO<byte[]> srcIo, long srcPageAddr,
+            int srcIdx) throws IgniteCheckedException {
+            storeByOffset(dstPageAddr, offset(dstIdx), srcIo.getLookupRow(null, srcPageAddr, srcIdx));
+        }
+
+        /** {@inheritDoc} */
+        @Override public byte[] getLookupRow(BPlusTree<byte[],?> tree, long pageAddr, int idx) {
+            return PageUtils.getBytes(pageAddr, offset(idx), itemSize);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/compress/src/test/java/org/apache/ignite/internal/processors/compress/DiskPageCompressionIntegrationAsyncTest.java
----------------------------------------------------------------------
diff --git a/modules/compress/src/test/java/org/apache/ignite/internal/processors/compress/DiskPageCompressionIntegrationAsyncTest.java b/modules/compress/src/test/java/org/apache/ignite/internal/processors/compress/DiskPageCompressionIntegrationAsyncTest.java
new file mode 100644
index 0000000..658a5d2
--- /dev/null
+++ b/modules/compress/src/test/java/org/apache/ignite/internal/processors/compress/DiskPageCompressionIntegrationAsyncTest.java
@@ -0,0 +1,30 @@
+/*
+ * 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.internal.processors.compress;
+
+import org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+
+/**
+ */
+public class DiskPageCompressionIntegrationAsyncTest extends DiskPageCompressionIntegrationTest {
+    /** {@inheritDoc} */
+    @Override protected FileIOFactory getFileIOFactory() {
+        return new AsyncFileIOFactory();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/compress/src/test/java/org/apache/ignite/internal/processors/compress/DiskPageCompressionIntegrationTest.java
----------------------------------------------------------------------
diff --git a/modules/compress/src/test/java/org/apache/ignite/internal/processors/compress/DiskPageCompressionIntegrationTest.java b/modules/compress/src/test/java/org/apache/ignite/internal/processors/compress/DiskPageCompressionIntegrationTest.java
new file mode 100644
index 0000000..ca7f4ea
--- /dev/null
+++ b/modules/compress/src/test/java/org/apache/ignite/internal/processors/compress/DiskPageCompressionIntegrationTest.java
@@ -0,0 +1,468 @@
+/*
+ * 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.internal.processors.compress;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.nio.file.OpenOption;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.DiskPageCompression;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIODecorator;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.mxbean.CacheGroupMetricsMXBean;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.configuration.DataStorageConfiguration.MAX_PAGE_SIZE;
+import static org.apache.ignite.configuration.DiskPageCompression.LZ4;
+import static org.apache.ignite.configuration.DiskPageCompression.SKIP_GARBAGE;
+import static org.apache.ignite.configuration.DiskPageCompression.SNAPPY;
+import static org.apache.ignite.configuration.DiskPageCompression.ZSTD;
+import static org.apache.ignite.internal.processors.compress.CompressionProcessor.LZ4_DEFAULT_LEVEL;
+import static org.apache.ignite.internal.processors.compress.CompressionProcessor.LZ4_MAX_LEVEL;
+import static org.apache.ignite.internal.processors.compress.CompressionProcessor.LZ4_MIN_LEVEL;
+import static org.apache.ignite.internal.processors.compress.CompressionProcessor.ZSTD_MAX_LEVEL;
+import static org.apache.ignite.internal.processors.compress.CompressionProcessor.ZSTD_MIN_LEVEL;
+
+/**
+ *
+ */
+public class DiskPageCompressionIntegrationTest extends GridCommonAbstractTest {
+    /** */
+    private DiskPageCompression compression;
+
+    /** */
+    private Integer compressionLevel;
+
+    /** */
+    private FileIOFactory factory;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        compression = null;
+        compressionLevel = null;
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteName) throws Exception {
+        DataRegionConfiguration drCfg = new DataRegionConfiguration()
+            .setPersistenceEnabled(true);
+
+        factory = getFileIOFactory();
+
+        DataStorageConfiguration dsCfg = new DataStorageConfiguration()
+            .setMetricsEnabled(true)
+            .setPageSize(MAX_PAGE_SIZE)
+            .setDefaultDataRegionConfiguration(drCfg)
+            .setFileIOFactory(U.isLinux() ? factory : new PunchFileIOFactory(factory));
+
+        return super.getConfiguration(igniteName).setDataStorageConfiguration(dsCfg);
+    }
+
+    /**
+     * @return File IO factory.
+     */
+    protected FileIOFactory getFileIOFactory() {
+        return new RandomAccessFileIOFactory();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPageCompression_Zstd_Max() throws Exception {
+        compression = ZSTD;
+        compressionLevel = ZSTD_MAX_LEVEL;
+
+        doTestPageCompression();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPageCompression_Zstd_Default() throws Exception {
+        compression = ZSTD;
+        compressionLevel = null;
+
+        doTestPageCompression();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPageCompression_Zstd_Min() throws Exception {
+        compression = ZSTD;
+        compressionLevel = ZSTD_MIN_LEVEL;
+
+        doTestPageCompression();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPageCompression_Lz4_Max() throws Exception {
+        compression = LZ4;
+        compressionLevel = LZ4_MAX_LEVEL;
+
+        doTestPageCompression();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPageCompression_Lz4_Default() throws Exception {
+        compression = LZ4;
+        compressionLevel = null;
+
+        doTestPageCompression();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPageCompression_Lz4_Min() throws Exception {
+        assertEquals(LZ4_MIN_LEVEL, LZ4_DEFAULT_LEVEL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPageCompression_SkipGarbage() throws Exception {
+        compression = SKIP_GARBAGE;
+
+        doTestPageCompression();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPageCompression_Snappy() throws Exception {
+        compression = SNAPPY;
+
+        doTestPageCompression();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void doTestPageCompression() throws Exception {
+        IgniteEx ignite = startGrid(0);
+
+        ignite.cluster().active(true);
+
+        String cacheName = "test";
+
+        CacheConfiguration<Integer,TestVal> ccfg = new CacheConfiguration<Integer,TestVal>()
+            .setName(cacheName)
+            .setBackups(0)
+            .setAtomicityMode(ATOMIC)
+            .setIndexedTypes(Integer.class, TestVal.class)
+            .setDiskPageCompression(compression)
+            .setDiskPageCompressionLevel(compressionLevel);
+
+        IgniteCache<Integer,TestVal> cache = ignite.getOrCreateCache(ccfg);
+
+        int cnt = 2_000;
+
+        for (int i = 0; i < cnt; i++)
+            assertTrue(cache.putIfAbsent(i, new TestVal(i)));
+
+        for (int i = 0; i < cnt; i += 2)
+            assertEquals(new TestVal(i), cache.getAndRemove(i));
+
+        GridCacheDatabaseSharedManager dbMgr = ((GridCacheDatabaseSharedManager)ignite.context()
+            .cache().context().database());
+
+        dbMgr.forceCheckpoint("test compression").finishFuture().get();
+
+        FilePageStoreManager storeMgr = dbMgr.getFileStoreManager();
+
+        checkFileIOFactory(storeMgr.getPageStoreFileIoFactory());
+
+        Thread.sleep(100); // Wait for metrics update.
+
+        long storeSize = ignite.dataStorageMetrics().getStorageSize();
+        long sparseStoreSize = ignite.dataStorageMetrics().getSparseStorageSize();
+
+        assertTrue("storeSize: " + storeSize, storeSize > 0);
+
+        if (U.isLinux()) {
+            assertTrue("sparseSize: " + sparseStoreSize, sparseStoreSize > 0);
+            assertTrue(storeSize + " > " + sparseStoreSize, storeSize > sparseStoreSize);
+        }
+        else
+            assertTrue(sparseStoreSize < 0);
+
+        GridCacheContext<?,?> cctx = ignite.cachex(cacheName).context();
+
+        int cacheId = cctx.cacheId();
+        int groupId = cctx.groupId();
+
+        assertEquals(cacheId, groupId);
+
+        CacheGroupMetricsMXBean mx = cctx.group().mxBean();
+
+        storeSize = mx.getStorageSize();
+        sparseStoreSize = mx.getSparseStorageSize();
+
+        assertTrue("storeSize: " + storeSize, storeSize > 0);
+
+        if (U.isLinux()) {
+            assertTrue("sparseSize: " + sparseStoreSize, sparseStoreSize > 0);
+            assertTrue(storeSize + " > " + sparseStoreSize, storeSize > sparseStoreSize);
+        }
+        else
+            assertTrue(sparseStoreSize < 0);
+
+        int parts = cctx.affinity().partitions();
+
+        for (int i = 0; i < parts; i++) {
+            PageStore store = storeMgr.getStore(cacheId, i);
+
+            long realSize = store.size();
+            long virtualSize = store.getPageSize() * store.pages();
+            long sparseSize = store.getSparseSize();
+
+            assertTrue(virtualSize > 0);
+
+            error("virt: " + virtualSize + ",  real: " + realSize + ",  sparse: " + sparseSize);
+
+            if (!store.exists())
+                continue;
+
+            if (virtualSize > sparseSize)
+                return;
+        }
+
+        fail("No files were compacted.");
+    }
+
+    /**
+     */
+    public void _testCompressionRatio() throws Exception {
+        IgniteEx ignite = startGrid(0);
+
+        ignite.cluster().active(true);
+
+        String cacheName = "test";
+
+        CacheConfiguration<Integer,TestVal> ccfg = new CacheConfiguration<Integer,TestVal>()
+            .setName(cacheName)
+            .setBackups(0)
+            .setAtomicityMode(ATOMIC)
+            .setIndexedTypes(Integer.class, TestVal.class)
+            .setAffinity(new RendezvousAffinityFunction().setPartitions(10))
+            .setDiskPageCompression(ZSTD);
+//            .setDiskPageCompressionLevel(compressionLevel);
+
+        ignite.getOrCreateCache(ccfg);
+
+        IgniteInternalCache<Integer,TestVal> cache = ignite.cachex(cacheName);
+
+        CacheGroupMetricsMXBean mx = cache.context().group().mxBean();
+
+        GridCacheDatabaseSharedManager dbMgr = ((GridCacheDatabaseSharedManager)ignite.context()
+            .cache().context().database());
+
+        int cnt = 20_000_000;
+
+        for (int i = 0; i < cnt; i++) {
+            assertTrue(cache.putIfAbsent(i, new TestVal(i)));
+
+            if (i % 50_000 == 0) {
+                dbMgr.forceCheckpoint("test").finishFuture().get();
+
+                long sparse = mx.getSparseStorageSize();
+                long size = mx.getStorageSize();
+
+                System.out.println(i + " >> " + sparse + " / " + size + " = " + ((double)sparse / size));
+            }
+        }
+    }
+
+    /**
+     * @param f Factory.
+     */
+    protected void checkFileIOFactory(FileIOFactory f) {
+        if (!U.isLinux())
+            f = ((PunchFileIOFactory)f).delegate;
+
+        assertSame(factory, f);
+    }
+
+    /**
+     */
+    static class TestVal implements Serializable {
+        /** */
+        static final long serialVersionUID = 1L;
+
+        /** */
+        @QuerySqlField
+        String str;
+
+        /** */
+        int i;
+
+        /** */
+        @QuerySqlField
+        long x;
+
+        /** */
+        @QuerySqlField
+        UUID id;
+
+        TestVal(int i) {
+            this.str =  i + "bla bla bla!";
+            this.i = -i;
+            this.x = 0xffaabbccdd773311L + i;
+            this.id = new UUID(i,-i);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            TestVal testVal = (TestVal)o;
+
+            if (i != testVal.i) return false;
+            if (x != testVal.x) return false;
+            if (str != null ? !str.equals(testVal.str) : testVal.str != null) return false;
+            return id != null ? id.equals(testVal.id) : testVal.id == null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int result = str != null ? str.hashCode() : 0;
+            result = 31 * result + i;
+            result = 31 * result + (int)(x ^ (x >>> 32));
+            result = 31 * result + (id != null ? id.hashCode() : 0);
+            return result;
+        }
+    }
+
+    /**
+     */
+    static class PunchFileIO extends FileIODecorator {
+        /** */
+        private ConcurrentMap<Long, Integer> holes = new ConcurrentHashMap<>();
+
+        /**
+         * @param delegate File I/O delegate
+         */
+        public PunchFileIO(FileIO delegate) {
+            super(Objects.requireNonNull(delegate));
+        }
+
+        /** {@inheritDoc} */
+        @Override public int getFileSystemBlockSize() {
+            assertFalse(U.isLinux());
+
+            return 4 * 1024;
+        }
+
+        /** {@inheritDoc} */
+        @Override public long getSparseSize() {
+            assertFalse(U.isLinux());
+
+            long holesSize = holes.values().stream().mapToLong(x -> x).sum();
+
+            try {
+                return size() - holesSize;
+            }
+            catch (IOException e) {
+                throw new IgniteException(e);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public int writeFully(ByteBuffer srcBuf, long position) throws IOException {
+            assertFalse(U.isLinux());
+
+            holes.remove(position);
+            return super.writeFully(srcBuf, position);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int punchHole(long pos, int len) {
+            assertFalse(U.isLinux());
+
+            assertTrue(len > 0);
+
+            int blockSize = getFileSystemBlockSize();
+
+            len = len / blockSize * blockSize;
+
+            if (len > 0)
+                holes.put(pos, len);
+
+            return len;
+        }
+    }
+
+    /**
+     */
+    static class PunchFileIOFactory implements FileIOFactory {
+        /** */
+        final FileIOFactory delegate;
+
+        /**
+         * @param delegate Delegate.
+         */
+        PunchFileIOFactory(FileIOFactory delegate) {
+            this.delegate = Objects.requireNonNull(delegate);
+        }
+
+        /** {@inheritDoc} */
+        @Override public FileIO create(File file) throws IOException {
+            return new PunchFileIO(delegate.create(file));
+        }
+
+        /** {@inheritDoc} */
+        @Override public FileIO create(File file, OpenOption... modes) throws IOException {
+            return new PunchFileIO(delegate.create(file, modes));
+        }
+    }
+}


[35/50] [abbrv] ignite git commit: IGNITE-9948 Fixed ettyRestProcessorAuthenticationWithTokenSelfTest.testGetOrCreate - Fixes #5038.

Posted by ag...@apache.org.
IGNITE-9948 Fixed ettyRestProcessorAuthenticationWithTokenSelfTest.testGetOrCreate - Fixes #5038.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-9720
Commit: 882b5a4a80dd1c49d4ff17818a07e38f4279ff3c
Parents: 9031733
Author: NSAmelchev <ns...@gmail.com>
Authored: Tue Nov 27 19:21:08 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Nov 27 19:21:08 2018 +0300

----------------------------------------------------------------------
 .../JettyRestProcessorAbstractSelfTest.java     | 90 ++++++++++----------
 ...RestProcessorAuthenticationAbstractTest.java |  6 +-
 2 files changed, 50 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/882b5a4a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
index 703f50d..81f0a36f 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.rest;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import java.io.IOException;
 import java.io.Serializable;
 import java.io.UnsupportedEncodingException;
@@ -34,9 +35,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
-
-import com.fasterxml.jackson.databind.JsonNode;
-
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
@@ -238,17 +236,21 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
     }
 
     /**
+     * Validates JSON response.
+     *
      * @param content Content to check.
      * @return REST result.
+     * @throws IOException If parsing failed.
      */
-    protected JsonNode jsonResponse(String content) throws IOException {
+    protected JsonNode validateJsonResponse(String content) throws IOException {
         assertNotNull(content);
         assertFalse(content.isEmpty());
 
         JsonNode node = JSON_MAPPER.readTree(content);
 
+        assertTrue("Unexpected error: " + node.get("error").asText(), node.get("error").isNull());
+
         assertEquals(STATUS_SUCCESS, node.get("successStatus").asInt());
-        assertTrue(node.get("error").isNull());
 
         assertNotSame(securityEnabled(), node.get("sessionToken").isNull());
 
@@ -341,7 +343,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
         // Test with SQL.
         SqlFieldsQuery qry = new SqlFieldsQuery(
             "create table employee(id integer primary key, name varchar(100), salary integer);" +
-            "insert into employee(id, name, salary) values (1, 'Alex', 300);"
+                "insert into employee(id, name, salary) values (1, 'Alex', 300);"
         );
 
         grid(0).context().query().querySqlFields(qry, true, false);
@@ -420,7 +422,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
 
         info("Get command result: " + ret);
 
-        JsonNode res = jsonResponse(ret);
+        JsonNode res = validateJsonResponse(ret);
 
         assertEquals(F.asMap("", null, "key", "value"), JSON_MAPPER.treeToValue(res, HashMap.class));
 
@@ -434,7 +436,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
 
         info("Get command result: " + ret);
 
-        res = jsonResponse(ret);
+        res = validateJsonResponse(ret);
 
         assertEquals(F.asMap("", "value", "key", null), JSON_MAPPER.treeToValue(res, HashMap.class));
     }
@@ -565,7 +567,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
 
         info("Name command result: " + ret);
 
-        assertEquals(getTestIgniteInstanceName(0), jsonResponse(ret).asText());
+        assertEquals(getTestIgniteInstanceName(0), validateJsonResponse(ret).asText());
     }
 
     /**
@@ -588,6 +590,8 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
 
         info("GetOrCreateCache command result: " + ret);
 
+        validateJsonResponse(ret);
+
         IgniteCache<String, String> cache = grid(0).cache(cacheName);
 
         cache.put("1", "1");
@@ -606,7 +610,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
 
         ret = content(cacheName, GridRestCommand.DESTROY_CACHE);
 
-        assertTrue(jsonResponse(ret).isNull());
+        assertTrue(validateJsonResponse(ret).isNull());
         assertNull(grid(0).cache(cacheName));
     }
 
@@ -1117,7 +1121,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
             "delta", "3"
         );
 
-        JsonNode res = jsonResponse(ret);
+        JsonNode res = validateJsonResponse(ret);
 
         assertEquals(5, res.asInt());
         assertEquals(5, grid(0).atomicLong("incrKey", 0, true).get());
@@ -1127,7 +1131,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
             "delta", "10"
         );
 
-        res = jsonResponse(ret);
+        res = validateJsonResponse(ret);
 
         assertEquals(15, res.asInt());
         assertEquals(15, grid(0).atomicLong("incrKey", 0, true).get());
@@ -1143,7 +1147,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
             "delta", "10"
         );
 
-        JsonNode res = jsonResponse(ret);
+        JsonNode res = validateJsonResponse(ret);
 
         assertEquals(5, res.asInt());
         assertEquals(5, grid(0).atomicLong("decrKey", 0, true).get());
@@ -1153,7 +1157,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
             "delta", "3"
         );
 
-        res = jsonResponse(ret);
+        res = validateJsonResponse(ret);
 
         assertEquals(2, res.asInt());
         assertEquals(2, grid(0).atomicLong("decrKey", 0, true).get());
@@ -1321,7 +1325,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
 
         info("Cache metadata: " + ret);
 
-        JsonNode arrRes = jsonResponse(ret);
+        JsonNode arrRes = validateJsonResponse(ret);
 
         // TODO: IGNITE-7740 uncomment after IGNITE-7740 will be fixed.
         // assertEquals(cachesCnt, arrRes.size());
@@ -1334,7 +1338,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
 
         info("Cache metadata: " + ret);
 
-        arrRes = jsonResponse(ret);
+        arrRes = validateJsonResponse(ret);
 
         assertEquals(1, arrRes.size());
 
@@ -1361,7 +1365,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
 
         info("Cache metadata: " + ret);
 
-        JsonNode arrRes = jsonResponse(ret);
+        JsonNode arrRes = validateJsonResponse(ret);
 
         // TODO: IGNITE-7740 uncomment after IGNITE-7740 will be fixed.
         // int cachesCnt = grid(1).cacheNames().size();
@@ -1373,7 +1377,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
 
         info("Cache metadata with cacheName parameter: " + ret);
 
-        arrRes = jsonResponse(ret);
+        arrRes = validateJsonResponse(ret);
 
         assertEquals(1, arrRes.size());
 
@@ -1394,7 +1398,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
 
         info("Topology command result: " + ret);
 
-        JsonNode res = jsonResponse(ret);
+        JsonNode res = validateJsonResponse(ret);
 
         assertEquals(gridCount(), res.size());
 
@@ -1438,7 +1442,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
 
         info("Topology command result: " + ret);
 
-        res = jsonResponse(ret);
+        res = validateJsonResponse(ret);
 
         assertEquals(gridCount(), res.size());
 
@@ -1461,7 +1465,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
 
         info("Topology command result: " + ret);
 
-        JsonNode res = jsonResponse(ret);
+        JsonNode res = validateJsonResponse(ret);
 
         assertTrue(res.get("attributes").isObject());
         assertTrue(res.get("metrics").isObject());
@@ -1480,7 +1484,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
 
         info("Topology command result: " + ret);
 
-        res = jsonResponse(ret);
+        res = validateJsonResponse(ret);
 
         assertTrue(res.get("attributes").isNull());
         assertTrue(res.get("metrics").isNull());
@@ -1494,7 +1498,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
 
         info("Topology command result: " + ret);
 
-        res = jsonResponse(ret);
+        res = validateJsonResponse(ret);
 
         assertTrue(res.isNull());
 
@@ -1508,7 +1512,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
 
         info("Topology command result: " + ret);
 
-        res = jsonResponse(ret);
+        res = validateJsonResponse(ret);
 
         assertTrue(res.get("attributes").isNull());
         assertTrue(res.get("metrics").isNull());
@@ -1927,7 +1931,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
     public void testVersion() throws Exception {
         String ret = content(null, GridRestCommand.VERSION);
 
-        JsonNode res = jsonResponse(ret);
+        JsonNode res = validateJsonResponse(ret);
 
         assertEquals(VER_STR, res.asText());
     }
@@ -1946,7 +1950,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
             "arg2", "2000"
         );
 
-        JsonNode items = jsonResponse(ret).get("items");
+        JsonNode items = validateJsonResponse(ret).get("items");
 
         assertEquals(2, items.size());
 
@@ -1962,7 +1966,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
             "cacheName", "person"
         );
 
-        JsonNode items = jsonResponse(ret).get("items");
+        JsonNode items = validateJsonResponse(ret).get("items");
 
         assertEquals(4, items.size());
 
@@ -1978,7 +1982,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
             "className", ScanFilter.class.getName()
         );
 
-        JsonNode items = jsonResponse(ret).get("items");
+        JsonNode items = validateJsonResponse(ret).get("items");
 
         assertEquals(2, items.size());
 
@@ -2013,18 +2017,18 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
             "qry", URLEncoder.encode("select * from String", CHARSET)
         );
 
-        JsonNode qryId = jsonResponse(ret).get("queryId");
+        JsonNode qryId = validateJsonResponse(ret).get("queryId");
 
-        assertFalse(jsonResponse(ret).get("queryId").isNull());
+        assertFalse(validateJsonResponse(ret).get("queryId").isNull());
 
         ret = content(DEFAULT_CACHE_NAME, GridRestCommand.FETCH_SQL_QUERY,
             "pageSize", "1",
             "qryId", qryId.asText()
         );
 
-        JsonNode res = jsonResponse(ret);
+        JsonNode res = validateJsonResponse(ret);
 
-        JsonNode qryId0 = jsonResponse(ret).get("queryId");
+        JsonNode qryId0 = validateJsonResponse(ret).get("queryId");
 
         assertEquals(qryId0, qryId);
         assertFalse(res.get("last").asBoolean());
@@ -2034,9 +2038,9 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
             "qryId", qryId.asText()
         );
 
-        res = jsonResponse(ret);
+        res = validateJsonResponse(ret);
 
-        qryId0 = jsonResponse(ret).get("queryId");
+        qryId0 = validateJsonResponse(ret).get("queryId");
 
         assertEquals(qryId0, qryId);
         assertTrue(res.get("last").asBoolean());
@@ -2060,7 +2064,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
             "arg1", "o1"
         );
 
-        JsonNode items = jsonResponse(ret).get("items");
+        JsonNode items = validateJsonResponse(ret).get("items");
 
         assertEquals(2, items.size());
 
@@ -2078,7 +2082,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
             "qry", URLEncoder.encode(qry, CHARSET)
         );
 
-        JsonNode items = jsonResponse(ret).get("items");
+        JsonNode items = validateJsonResponse(ret).get("items");
 
         assertEquals(4, items.size());
 
@@ -2097,7 +2101,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
             "qry", URLEncoder.encode(qry, CHARSET)
         );
 
-        JsonNode items = jsonResponse(ret).get("items");
+        JsonNode items = validateJsonResponse(ret).get("items");
 
         assertEquals(4, items.size());
 
@@ -2115,7 +2119,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
             "qry", URLEncoder.encode(qry, CHARSET)
         );
 
-        JsonNode res = jsonResponse(ret);
+        JsonNode res = validateJsonResponse(ret);
 
         JsonNode items = res.get("items");
 
@@ -2148,7 +2152,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
             "arg2", "2000"
         );
 
-        JsonNode res = jsonResponse(ret);
+        JsonNode res = validateJsonResponse(ret);
 
         assertEquals(1, res.get("items").size());
 
@@ -2180,7 +2184,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
                 "arg2", "2000"
             );
 
-        JsonNode items = jsonResponse(ret).get("items");
+        JsonNode items = validateJsonResponse(ret).get("items");
 
         assertEquals(1, items.size());
 
@@ -2397,7 +2401,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
 
         info("Command result: " + ret);
 
-        JsonNode json = jsonResponse(ret);
+        JsonNode json = validateJsonResponse(ret);
 
         assertEquals(exp, json.isObject() ? json.toString() : json.asText());
     }
@@ -3027,7 +3031,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
         String ret = content("cmd", GridRestCommand.CLUSTER_CURRENT_STATE);
 
         info("Cluster state: " + ret);
-        JsonNode res = jsonResponse(ret);
+        JsonNode res = validateJsonResponse(ret);
 
         assertEquals(exp, res.asBoolean());
         assertEquals(exp, grid(0).cluster().active());
@@ -3042,7 +3046,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends JettyRestProces
     private void changeClusterState(GridRestCommand cmd) throws Exception {
         String ret = content(null, cmd);
 
-        JsonNode res = jsonResponse(ret);
+        JsonNode res = validateJsonResponse(ret);
 
         assertFalse(res.isNull());
         assertTrue(res.asText().startsWith(cmd.key()));

http://git-wip-us.apache.org/repos/asf/ignite/blob/882b5a4a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationAbstractTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationAbstractTest.java
index 0ed9e95..8f99cdb 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationAbstractTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationAbstractTest.java
@@ -105,7 +105,7 @@ public abstract class JettyRestProcessorAuthenticationAbstractTest extends Jetty
             "user", "user1",
             "password", "password1");
 
-        JsonNode res = jsonResponse(ret);
+        JsonNode res = validateJsonResponse(ret);
 
         assertTrue(res.asBoolean());
 
@@ -118,7 +118,7 @@ public abstract class JettyRestProcessorAuthenticationAbstractTest extends Jetty
             "user", "user1",
             "password", "password2");
 
-        res = jsonResponse(ret);
+        res = validateJsonResponse(ret);
 
         assertTrue(res.asBoolean());
 
@@ -128,7 +128,7 @@ public abstract class JettyRestProcessorAuthenticationAbstractTest extends Jetty
         ret = content(null, GridRestCommand.REMOVE_USER,
             "user", "user1");
 
-        res = jsonResponse(ret);
+        res = validateJsonResponse(ret);
 
         assertTrue(res.asBoolean());
 


[28/50] [abbrv] ignite git commit: IGNITE-10002: MVCC: Create "Cache 2" test suite for MVCC mode. This closes #5198.

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalIsolatedNodesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalIsolatedNodesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalIsolatedNodesSelfTest.java
index 53122c7..e5d3b48 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalIsolatedNodesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalIsolatedNodesSelfTest.java
@@ -23,6 +23,7 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheMode.LOCAL;
@@ -31,6 +32,13 @@ import static org.apache.ignite.cache.CacheMode.LOCAL;
  * Isolated nodes LOCAL cache self test.
  */
 public class GridCacheLocalIsolatedNodesSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.LOCAL_CACHE);
+
+        super.beforeTestsStarted();
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalLoadAllSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalLoadAllSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalLoadAllSelfTest.java
index f10cefd..647027f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalLoadAllSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalLoadAllSelfTest.java
@@ -27,6 +27,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheMode.LOCAL;
@@ -35,6 +36,13 @@ import static org.apache.ignite.cache.CacheMode.LOCAL;
  * Load-All self test.
  */
 public class GridCacheLocalLoadAllSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.LOCAL_CACHE);
+
+        super.setUp();
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalLockSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalLockSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalLockSelfTest.java
index c7809df..8f27b1b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalLockSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalLockSelfTest.java
@@ -28,6 +28,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestThread;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
@@ -37,6 +38,13 @@ import static org.apache.ignite.cache.CacheMode.LOCAL;
  * Test cases for multi-threaded tests.
  */
 public class GridCacheLocalLockSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.LOCAL_CACHE);
+
+        super.setUp();
+    }
+
     /** Grid. */
     private Ignite ignite;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalMultithreadedSelfTest.java
index f6dc535..4911631 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalMultithreadedSelfTest.java
@@ -34,6 +34,7 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestThread;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheMode.LOCAL;
@@ -42,6 +43,13 @@ import static org.apache.ignite.cache.CacheMode.LOCAL;
  * Multithreaded local cache locking test.
  */
 public class GridCacheLocalMultithreadedSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.LOCAL_CACHE);
+
+        super.setUp();
+    }
+
     /** Cache. */
     private IgniteCache<Integer, String> cache;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxMultiThreadedSelfTest.java
index 6776550..816168f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxMultiThreadedSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.cache.IgniteTxMultiThreadedAbstractTest;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 
@@ -34,6 +35,14 @@ public class GridCacheLocalTxMultiThreadedSelfTest extends IgniteTxMultiThreaded
     private static final boolean CACHE_DEBUG = false;
 
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.LOCAL_CACHE);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings({"ConstantConditions"})
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxSingleThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxSingleThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxSingleThreadedSelfTest.java
index 22809c7..27ce919 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxSingleThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxSingleThreadedSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.cache.IgniteTxSingleThreadedAbstractTest;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 
@@ -30,6 +31,13 @@ import static org.apache.ignite.cache.CacheMode.LOCAL;
  * Tests for local transactions.
  */
 public class GridCacheLocalTxSingleThreadedSelfTest extends IgniteTxSingleThreadedAbstractTest {
+    /** {@inheritDoc} */
+    @Override public void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.LOCAL_CACHE);
+
+        super.beforeTestsStarted();
+    }
+
     /** Cache debug flag. */
     private static final boolean CACHE_DEBUG = false;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxTimeoutSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxTimeoutSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxTimeoutSelfTest.java
index 160e251..62af452 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxTimeoutSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxTimeoutSelfTest.java
@@ -25,6 +25,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -42,6 +43,13 @@ import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE;
  *
  */
 public class GridCacheLocalTxTimeoutSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.LOCAL_CACHE);
+
+        super.setUp();
+    }
+
     /** Grid. */
     private Ignite ignite;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/testframework/MvccFeatureChecker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/MvccFeatureChecker.java b/modules/core/src/test/java/org/apache/ignite/testframework/MvccFeatureChecker.java
new file mode 100644
index 0000000..ee9b02a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/MvccFeatureChecker.java
@@ -0,0 +1,137 @@
+/*
+ * 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.testframework;
+
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_FORCE_MVCC_MODE_IN_TESTS;
+import static org.junit.Assert.fail;
+
+/**
+ * Provides checks for features supported when FORCE_MVCC mode is on.
+ */
+public class MvccFeatureChecker {
+    /** */
+    private static final boolean FORCE_MVCC =
+        IgniteSystemProperties.getBoolean(IGNITE_FORCE_MVCC_MODE_IN_TESTS, false);
+
+    /** */
+    public enum Feature {
+        CACHE_STORE,
+        NEAR_CACHE,
+        LOCAL_CACHE,
+        ENTRY_LOCK,
+        CACHE_EVENTS,
+        EVICTION,
+        EXPIRATION
+    }
+
+    /**
+     * Fails if feature is not supported.
+     *
+     * @param f feature.
+     * @throws AssertionError If failed.
+     */
+    public static void failIfNotSupported(Feature f) {
+        if (!forcedMvcc())
+            return;
+
+        validateFeature(f);
+    }
+
+    /**
+     * @return {@code True} if Mvcc mode is forced.
+     */
+    public static boolean forcedMvcc() {
+        return FORCE_MVCC;
+    }
+
+    /**
+     * Check if feature is supported.
+     *
+     * @param f Feature.
+     * @return {@code True} if feature is supported, {@code False} otherwise.
+     */
+    public static boolean isSupported(Feature f) {
+        try {
+            validateFeature(f);
+
+            return true;
+        }
+        catch (AssertionError ignore) {
+            return false;
+        }
+    }
+
+    /**
+     * Check if Tx mode is supported.
+     *
+     * @param conc Transaction concurrency.
+     * @param iso Transaction isolation.
+     * @return {@code True} if feature is supported, {@code False} otherwise.
+     */
+    public static boolean isSupported(TransactionConcurrency conc, TransactionIsolation iso) {
+            return conc == TransactionConcurrency.PESSIMISTIC &&
+                iso == TransactionIsolation.REPEATABLE_READ;
+    }
+
+
+    /**
+     * Check if Cache mode is supported.
+     *
+     * @param mode Cache mode.
+     * @return {@code True} if feature is supported, {@code False} otherwise.
+     */
+    public static boolean isSupported(CacheMode mode) {
+        return mode != CacheMode.LOCAL || isSupported(Feature.LOCAL_CACHE);
+    }
+
+    /**
+     * Fails if feature is not supported in Mvcc mode.
+     *
+     * @param feature Mvcc feature.
+     * @throws AssertionError If failed.
+     */
+    private static void validateFeature(Feature feature) {
+        switch (feature) {
+            case NEAR_CACHE:
+                fail("https://issues.apache.org/jira/browse/IGNITE-7187");
+
+            case LOCAL_CACHE:
+                fail("https://issues.apache.org/jira/browse/IGNITE-9530");
+
+            case CACHE_STORE:
+                fail("https://issues.apache.org/jira/browse/IGNITE-8582");
+
+            case ENTRY_LOCK:
+                fail("https://issues.apache.org/jira/browse/IGNITE-9324");
+
+            case CACHE_EVENTS:
+                fail("https://issues.apache.org/jira/browse/IGNITE-9321");
+
+            case EVICTION:
+                fail("https://issues.apache.org/jira/browse/IGNITE-7956");
+
+            case EXPIRATION:
+                fail("https://issues.apache.org/jira/browse/IGNITE-7311");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite2.java
new file mode 100644
index 0000000..e0d1eef
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite2.java
@@ -0,0 +1,197 @@
+/*
+ * 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.testsuites;
+
+import java.util.HashSet;
+import junit.framework.TestSuite;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.affinity.rendezvous.ClusterNodeAttributeAffinityBackupFilterSelfTest;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunctionBackupFilterSelfTest;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunctionExcludeNeighborsSelfTest;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunctionFastPowerOfTwoHashSelfTest;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunctionSelfTest;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunctionStandardHashSelfTest;
+import org.apache.ignite.internal.IgniteReflectionFactorySelfTest;
+import org.apache.ignite.internal.processors.cache.CacheComparatorTest;
+import org.apache.ignite.internal.processors.cache.CacheConfigurationLeakTest;
+import org.apache.ignite.internal.processors.cache.CacheEnumOperationsSingleNodeTest;
+import org.apache.ignite.internal.processors.cache.CacheEnumOperationsTest;
+import org.apache.ignite.internal.processors.cache.CacheExchangeMessageDuplicatedStateTest;
+import org.apache.ignite.internal.processors.cache.CacheGroupLocalConfigurationSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheOptimisticTransactionsWithFilterSingleServerTest;
+import org.apache.ignite.internal.processors.cache.CacheOptimisticTransactionsWithFilterTest;
+import org.apache.ignite.internal.processors.cache.GridCacheAtomicMessageCountSelfTest;
+import org.apache.ignite.internal.processors.cache.GridCachePartitionedProjectionAffinitySelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteAtomicCacheEntryProcessorNodeJoinTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheNoSyncForGetTest;
+import org.apache.ignite.internal.processors.cache.IgniteCachePartitionMapUpdateTest;
+import org.apache.ignite.internal.processors.cache.IgniteClientCacheStartFailoverTest;
+import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheAndNodeStop;
+import org.apache.ignite.internal.processors.cache.IgniteNearClientCacheCloseTest;
+import org.apache.ignite.internal.processors.cache.IgniteOnePhaseCommitInvokeTest;
+import org.apache.ignite.internal.processors.cache.IgniteOnePhaseCommitNearReadersTest;
+import org.apache.ignite.internal.processors.cache.MemoryPolicyConfigValidationTest;
+import org.apache.ignite.internal.processors.cache.NonAffinityCoordinatorDynamicStartStopTest;
+import org.apache.ignite.internal.processors.cache.distributed.CacheLoadingConcurrentGridStartSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.CacheLoadingConcurrentGridStartSelfTestAllowOverwrite;
+import org.apache.ignite.internal.processors.cache.distributed.CachePartitionStateTest;
+import org.apache.ignite.internal.processors.cache.distributed.GridCachePartitionedNearDisabledMvccTxMultiThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.GridCachePartitionedNearDisabledTxMultiThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.GridCacheTransformEventSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheClientNodePartitionsExchangeTest;
+import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheServerNodeConcurrentStart;
+import org.apache.ignite.internal.processors.cache.distributed.dht.CachePartitionPartialCountersMapSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheColocatedMvccTxSingleThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheColocatedOptimisticTransactionSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheColocatedTxSingleThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheDhtAtomicEvictionNearReadersSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheDhtPreloadOnheapSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedUnloadEventsSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePartitionedBackupNodeFailureRecoveryTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicNearEvictionEventSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicNearMultiNodeSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicNearReadersSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearClientHitTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearJobExecutionSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearMultiGetSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearTxForceKeyTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedAffinitySelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedAtomicGetAndTransformStoreSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiThreadedPutGetSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMvccTxMultiThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMvccTxSingleThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMvccTxTimeoutSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedTxMultiThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedTxSingleThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedTxTimeoutSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheRendezvousAffinityClientSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.NearCacheSyncUpdateTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.NoneRebalanceModeSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedJobExecutionTest;
+import org.apache.ignite.internal.processors.cache.local.GridCacheLocalAtomicBasicStoreSelfTest;
+import org.apache.ignite.internal.processors.cache.local.GridCacheLocalAtomicGetAndTransformStoreSelfTest;
+import org.apache.ignite.internal.processors.cache.persistence.MemoryPolicyInitializationTest;
+import org.apache.ignite.internal.processors.continuous.IgniteNoCustomEventsOnNodeStart;
+
+/**
+ * Test suite.
+ */
+public class IgniteCacheMvccTestSuite2 extends TestSuite {
+    /**
+     * @return IgniteCache test suite.
+     */
+    public static TestSuite suite() {
+        System.setProperty(IgniteSystemProperties.IGNITE_FORCE_MVCC_MODE_IN_TESTS, "true");
+
+        HashSet<Class> ignoredTests = new HashSet<>(128);
+
+        // Skip classes that already contains Mvcc tests
+        ignoredTests.add(GridCacheTransformEventSelfTest.class);
+        ignoredTests.add(IgniteClientCacheStartFailoverTest.class);
+        ignoredTests.add(IgniteNearClientCacheCloseTest.class);
+        ignoredTests.add(IgniteCacheNoSyncForGetTest.class);
+        ignoredTests.add(CacheEnumOperationsSingleNodeTest.class);
+        ignoredTests.add(CacheEnumOperationsTest.class);
+        ignoredTests.add(NearCacheSyncUpdateTest.class);
+        ignoredTests.add(GridCacheNearMultiGetSelfTest.class);
+
+        // Optimistic tx tests.
+        ignoredTests.add(GridCacheColocatedOptimisticTransactionSelfTest.class);
+        ignoredTests.add(CacheOptimisticTransactionsWithFilterSingleServerTest.class);
+        ignoredTests.add(CacheOptimisticTransactionsWithFilterTest.class);
+
+        // Irrelevant Tx tests.
+        ignoredTests.add(IgniteOnePhaseCommitInvokeTest.class);
+        ignoredTests.add(IgniteOnePhaseCommitNearReadersTest.class);
+        ignoredTests.add(GridCacheDhtPreloadOnheapSelfTest.class);
+        ignoredTests.add(GridCachePartitionedMultiThreadedPutGetSelfTest.class); // On-heap test.
+
+        // Atomic cache tests.
+        ignoredTests.add(GridCacheLocalAtomicBasicStoreSelfTest.class);
+        ignoredTests.add(GridCacheLocalAtomicGetAndTransformStoreSelfTest.class);
+        ignoredTests.add(GridCacheAtomicNearMultiNodeSelfTest.class);
+        ignoredTests.add(GridCacheAtomicNearReadersSelfTest.class);
+        ignoredTests.add(GridCachePartitionedAtomicGetAndTransformStoreSelfTest.class);
+        ignoredTests.add(GridCacheAtomicNearEvictionEventSelfTest.class);
+        ignoredTests.add(GridCacheAtomicMessageCountSelfTest.class);
+        ignoredTests.add(IgniteAtomicCacheEntryProcessorNodeJoinTest.class);
+        ignoredTests.add(GridCacheDhtAtomicEvictionNearReadersSelfTest.class);
+        ignoredTests.add(GridCacheNearClientHitTest.class);
+        ignoredTests.add(GridCacheNearTxForceKeyTest.class);
+        ignoredTests.add(CacheLoadingConcurrentGridStartSelfTest.class);
+        ignoredTests.add(CacheLoadingConcurrentGridStartSelfTestAllowOverwrite.class);
+        ignoredTests.add(IgniteCachePartitionedBackupNodeFailureRecoveryTest.class);
+
+        // Other non-tx tests.
+        ignoredTests.add(RendezvousAffinityFunctionSelfTest.class);
+        ignoredTests.add(RendezvousAffinityFunctionExcludeNeighborsSelfTest.class);
+        ignoredTests.add(RendezvousAffinityFunctionFastPowerOfTwoHashSelfTest.class);
+        ignoredTests.add(RendezvousAffinityFunctionStandardHashSelfTest.class);
+        ignoredTests.add(GridCachePartitionedAffinitySelfTest.class);
+        ignoredTests.add(GridCacheRendezvousAffinityClientSelfTest.class);
+        ignoredTests.add(GridCachePartitionedProjectionAffinitySelfTest.class);
+        ignoredTests.add(RendezvousAffinityFunctionBackupFilterSelfTest.class);
+        ignoredTests.add(ClusterNodeAttributeAffinityBackupFilterSelfTest.class);
+        ignoredTests.add(NonAffinityCoordinatorDynamicStartStopTest.class);
+
+        ignoredTests.add(NoneRebalanceModeSelfTest.class);
+        ignoredTests.add(IgniteCachePartitionMapUpdateTest.class);
+        ignoredTests.add(IgniteCacheClientNodePartitionsExchangeTest.class);
+        ignoredTests.add(IgniteCacheServerNodeConcurrentStart.class);
+
+        ignoredTests.add(GridCachePartitionedUnloadEventsSelfTest.class);
+
+        ignoredTests.add(IgniteNoCustomEventsOnNodeStart.class);
+        ignoredTests.add(CacheExchangeMessageDuplicatedStateTest.class);
+        ignoredTests.add(IgniteDynamicCacheAndNodeStop.class);
+
+        ignoredTests.add(GridCacheReplicatedJobExecutionTest.class);
+        ignoredTests.add(GridCacheNearJobExecutionSelfTest.class);
+
+        ignoredTests.add(CacheConfigurationLeakTest.class);
+        ignoredTests.add(MemoryPolicyConfigValidationTest.class);
+        ignoredTests.add(MemoryPolicyInitializationTest.class);
+        ignoredTests.add(CacheGroupLocalConfigurationSelfTest.class);
+
+        ignoredTests.add(CachePartitionStateTest.class);
+        ignoredTests.add(CacheComparatorTest.class);
+        ignoredTests.add(CachePartitionPartialCountersMapSelfTest.class);
+        ignoredTests.add(IgniteReflectionFactorySelfTest.class);
+
+        // Skip classes which Mvcc implementations are added in this method below.
+        // TODO IGNITE-10175: refactor these tests (use assume) to support both mvcc and non-mvcc modes after moving to JUnit4/5.
+        ignoredTests.add(GridCachePartitionedTxSingleThreadedSelfTest.class); // See GridCachePartitionedMvccTxSingleThreadedSelfTest
+        ignoredTests.add(GridCacheColocatedTxSingleThreadedSelfTest.class); // See GridCacheColocatedMvccTxSingleThreadedSelfTest
+        ignoredTests.add(GridCachePartitionedTxMultiThreadedSelfTest.class); // See GridCachePartitionedMvccTxMultiThreadedSelfTest
+        ignoredTests.add(GridCachePartitionedNearDisabledTxMultiThreadedSelfTest.class); // See GridCachePartitionedNearDisabledMvccTxMultiThreadedSelfTest
+        ignoredTests.add(GridCachePartitionedTxTimeoutSelfTest.class); // See GridCachePartitionedMvccTxTimeoutSelfTest
+
+        TestSuite suite = new TestSuite("IgniteCache Mvcc Test Suite part 2");
+
+        suite.addTest(IgniteCacheTestSuite2.suite(ignoredTests));
+
+        // Add Mvcc clones.
+        suite.addTestSuite(GridCachePartitionedMvccTxSingleThreadedSelfTest.class);
+        suite.addTestSuite(GridCacheColocatedMvccTxSingleThreadedSelfTest.class);
+        suite.addTestSuite(GridCachePartitionedMvccTxMultiThreadedSelfTest.class);
+        suite.addTestSuite(GridCachePartitionedNearDisabledMvccTxMultiThreadedSelfTest.class);
+        suite.addTestSuite(GridCachePartitionedMvccTxTimeoutSelfTest.class);
+
+        return suite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index fe09c07..179f5e9 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.testsuites;
 
+import java.util.HashSet;
 import junit.framework.TestSuite;
 import org.apache.ignite.cache.affinity.rendezvous.ClusterNodeAttributeAffinityBackupFilterSelfTest;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunctionBackupFilterSelfTest;
@@ -144,6 +145,7 @@ import org.apache.ignite.internal.processors.cache.local.GridCacheLocalTxSingleT
 import org.apache.ignite.internal.processors.cache.local.GridCacheLocalTxTimeoutSelfTest;
 import org.apache.ignite.internal.processors.cache.persistence.MemoryPolicyInitializationTest;
 import org.apache.ignite.internal.processors.continuous.IgniteNoCustomEventsOnNodeStart;
+import org.apache.ignite.testframework.GridTestUtils;
 
 /**
  * Test suite.
@@ -153,180 +155,197 @@ public class IgniteCacheTestSuite2 extends TestSuite {
      * @return IgniteCache test suite.
      * @throws Exception Thrown in case of the failure.
      */
-    public static TestSuite suite() throws Exception {
+    public static TestSuite suite() {
+        return suite(null);
+    }
+
+    /**
+     * @param ignoredTests Ignored tests.
+     * @return IgniteCache test suite.
+     * @throws Exception Thrown in case of the failure.
+     */
+    public static TestSuite suite(HashSet<Class> ignoredTests) {
         TestSuite suite = new TestSuite("IgniteCache Test Suite part 2");
 
         // Local cache.
-        suite.addTestSuite(GridCacheLocalBasicApiSelfTest.class);
-        suite.addTestSuite(GridCacheLocalBasicStoreSelfTest.class);
-        //suite.addTestSuite(GridCacheLocalBasicStoreMultithreadedSelfTest.class);
-        suite.addTestSuite(GridCacheLocalAtomicBasicStoreSelfTest.class);
-        suite.addTestSuite(GridCacheLocalGetAndTransformStoreSelfTest.class);
-        suite.addTestSuite(GridCacheLocalAtomicGetAndTransformStoreSelfTest.class);
-        suite.addTestSuite(GridCacheLocalLoadAllSelfTest.class);
-        suite.addTestSuite(GridCacheLocalLockSelfTest.class);
-        suite.addTestSuite(GridCacheLocalMultithreadedSelfTest.class);
-        suite.addTestSuite(GridCacheLocalTxSingleThreadedSelfTest.class);
-        //suite.addTestSuite(GridCacheLocalTxReadTest.class);
-        suite.addTestSuite(GridCacheLocalTxTimeoutSelfTest.class);
-        suite.addTestSuite(GridCacheLocalEventSelfTest.class);
-        suite.addTestSuite(GridCacheLocalEvictionEventSelfTest.class);
-        suite.addTestSuite(GridCacheVariableTopologySelfTest.class);
-        suite.addTestSuite(GridCacheLocalTxMultiThreadedSelfTest.class);
-        suite.addTestSuite(GridCacheTransformEventSelfTest.class);
-        suite.addTestSuite(GridCacheLocalIsolatedNodesSelfTest.class);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheLocalBasicApiSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheLocalBasicStoreSelfTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite,GridCacheLocalBasicStoreMultithreadedSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheLocalAtomicBasicStoreSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheLocalGetAndTransformStoreSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheLocalAtomicGetAndTransformStoreSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheLocalLoadAllSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheLocalLockSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheLocalMultithreadedSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheLocalTxSingleThreadedSelfTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite,GridCacheLocalTxReadTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheLocalTxTimeoutSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheLocalEventSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheLocalEvictionEventSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheLocalTxMultiThreadedSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheLocalIsolatedNodesSelfTest.class, ignoredTests);
+
+        GridTestUtils.addTestIfNeeded(suite, GridCacheTransformEventSelfTest.class, ignoredTests);
 
         // Partitioned cache.
-        suite.addTestSuite(GridCachePartitionedGetSelfTest.class);
-        suite.addTest(new TestSuite(GridCachePartitionedBasicApiTest.class));
-        suite.addTest(new TestSuite(GridCacheNearMultiGetSelfTest.class));
-        suite.addTest(new TestSuite(NoneRebalanceModeSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheNearJobExecutionSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheReplicatedJobExecutionTest.class));
-        suite.addTest(new TestSuite(GridCacheNearOneNodeSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheNearMultiNodeSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheAtomicNearMultiNodeSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheNearReadersSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheNearReaderPreloadSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheAtomicNearReadersSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedAffinitySelfTest.class));
-        //suite.addTest(new TestSuite(RendezvousAffinityFunctionSelfTest.class));
-        suite.addTest(new TestSuite(RendezvousAffinityFunctionExcludeNeighborsSelfTest.class));
-        suite.addTest(new TestSuite(RendezvousAffinityFunctionFastPowerOfTwoHashSelfTest.class));
-        suite.addTest(new TestSuite(RendezvousAffinityFunctionStandardHashSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheRendezvousAffinityClientSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedProjectionAffinitySelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedBasicOpSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedBasicStoreSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedGetAndTransformStoreSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedAtomicGetAndTransformStoreSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedBasicStoreMultiNodeSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedNearDisabledBasicStoreMultiNodeSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedEventSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedLockSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedNearDisabledLockSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedMultiNodeLockSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedMultiNodeSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedMultiThreadedPutGetSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedNodeFailureSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedExplicitLockNodeFailureSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedTxSingleThreadedSelfTest.class));
-        //suite.addTest(new TestSuite(GridCachePartitionedEntryLockSelfTest.class));
-        //suite.addTest(new TestSuite(GridCachePartitionedEvictionSelfTest.class));
-        //suite.addTest(new TestSuite(GridCachePartitionedNestedTxTest.class));
-        //suite.addTest(new TestSuite(GridCachePartitionedStorePutSelfTest.class));
-        //suite.addTest(new TestSuite(GridCachePartitionedTxConcurrentGetTest.class));
-        //suite.addTest(new TestSuite(GridCachePartitionedTxMultiNodeSelfTest.class));
-        //suite.addTest(new TestSuite(GridCachePartitionedTxReadTest.class));
-        suite.addTest(new TestSuite(GridCacheColocatedTxSingleThreadedSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedTxTimeoutSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheFinishPartitionsSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheDhtEntrySelfTest.class));
-        suite.addTest(new TestSuite(GridCacheDhtMappingSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedTxMultiThreadedSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedNearDisabledTxMultiThreadedSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheDhtPreloadSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheDhtPreloadOnheapSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheDhtPreloadBigDataSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheDhtPreloadPutGetSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheDhtPreloadDisabledSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheDhtPreloadMultiThreadedSelfTest.class));
-        suite.addTest(new TestSuite(CacheDhtLocalPartitionAfterRemoveSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheColocatedPreloadRestartSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheNearPreloadRestartSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheDhtPreloadStartStopSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheDhtPreloadUnloadSelfTest.class));
-        suite.addTest(new TestSuite(RendezvousAffinityFunctionBackupFilterSelfTest.class));
-        suite.addTest(new TestSuite(ClusterNodeAttributeAffinityBackupFilterSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedPreloadLifecycleSelfTest.class));
-        suite.addTest(new TestSuite(CacheLoadingConcurrentGridStartSelfTest.class));
-        suite.addTest(new TestSuite(CacheLoadingConcurrentGridStartSelfTestAllowOverwrite.class));
-        suite.addTest(new TestSuite(CacheTxLoadingConcurrentGridStartSelfTestAllowOverwrite.class));
-        suite.addTest(new TestSuite(GridCacheDhtPreloadDelayedSelfTest.class));
-        suite.addTest(new TestSuite(GridPartitionedBackupLoadSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedLoadCacheSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionNotLoadedEventSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheDhtEvictionsDisabledSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheNearEvictionEventSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheAtomicNearEvictionEventSelfTest.class));
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedGetSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedBasicApiTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedBasicOpSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheNearMultiGetSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, NoneRebalanceModeSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheNearOneNodeSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheNearMultiNodeSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheAtomicNearMultiNodeSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheNearReadersSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheNearReaderPreloadSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheAtomicNearReadersSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedGetAndTransformStoreSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedAtomicGetAndTransformStoreSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedBasicStoreSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridNearCacheStoreUpdateTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite,GridCachePartitionedStorePutSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedBasicStoreMultiNodeSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedNearDisabledBasicStoreMultiNodeSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, CacheConcurrentReadThroughTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedLockSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedNearDisabledLockSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedMultiNodeLockSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedMultiNodeSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedMultiThreadedPutGetSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedNodeFailureSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedExplicitLockNodeFailureSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, CacheLockReleaseNodeLeaveTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite,GridCachePartitionedEntryLockSelfTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite,GridCachePartitionedNestedTxTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite,GridCachePartitionedTxConcurrentGetTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite,GridCachePartitionedTxMultiNodeSelfTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite,GridCachePartitionedTxReadTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedTxSingleThreadedSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheColocatedTxSingleThreadedSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedTxTimeoutSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheFinishPartitionsSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedTxMultiThreadedSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedNearDisabledTxMultiThreadedSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheDhtEntrySelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheDhtMappingSelfTest.class, ignoredTests);
+
+        // Preload
+        GridTestUtils.addTestIfNeeded(suite, GridCacheDhtPreloadSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheDhtPreloadOnheapSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheDhtPreloadBigDataSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheDhtPreloadPutGetSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheDhtPreloadDisabledSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheDhtPreloadMultiThreadedSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheColocatedPreloadRestartSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheNearPreloadRestartSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheDhtPreloadStartStopSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheDhtPreloadUnloadSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedPreloadLifecycleSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheDhtPreloadDelayedSelfTest.class, ignoredTests);
+
+        GridTestUtils.addTestIfNeeded(suite, CacheDhtLocalPartitionAfterRemoveSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, CacheLoadingConcurrentGridStartSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, CacheLoadingConcurrentGridStartSelfTestAllowOverwrite.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, CacheTxLoadingConcurrentGridStartSelfTestAllowOverwrite.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridPartitionedBackupLoadSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedLoadCacheSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedEventSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionNotLoadedEventSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheDhtEvictionsDisabledSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheNearEvictionEventSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheAtomicNearEvictionEventSelfTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite,GridCachePartitionedEvictionSelfTest.class, ignoredTests);
+
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedTopologyChangeSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedUnloadEventsSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheColocatedOptimisticTransactionSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheAtomicMessageCountSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheNearPartitionedClearSelfTest.class, ignoredTests);
 
-        suite.addTest(new TestSuite(GridCachePartitionedTopologyChangeSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedUnloadEventsSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheColocatedOptimisticTransactionSelfTest.class));
-        suite.addTestSuite(GridCacheAtomicMessageCountSelfTest.class);
-        suite.addTest(new TestSuite(GridCacheNearPartitionedClearSelfTest.class));
+        GridTestUtils.addTestIfNeeded(suite, GridCacheOffheapUpdateSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheNearClientHitTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheNearPrimarySyncSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheColocatedPrimarySyncSelfTest.class, ignoredTests);
 
-        suite.addTest(new TestSuite(GridCacheOffheapUpdateSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheNearClientHitTest.class));
-        suite.addTest(new TestSuite(GridCacheNearPrimarySyncSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheColocatedPrimarySyncSelfTest.class));
+        GridTestUtils.addTestIfNeeded(suite, IgniteCachePartitionMapUpdateTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, IgniteCacheClientNodePartitionsExchangeTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, IgniteCacheClientNodeChangingTopologyTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, IgniteCacheServerNodeConcurrentStart.class, ignoredTests);
 
-        suite.addTest(new TestSuite(IgniteCachePartitionMapUpdateTest.class));
-        suite.addTest(new TestSuite(IgniteCacheClientNodePartitionsExchangeTest.class));
-        suite.addTest(new TestSuite(IgniteCacheClientNodeChangingTopologyTest.class));
-        suite.addTest(new TestSuite(IgniteCacheServerNodeConcurrentStart.class));
+        GridTestUtils.addTestIfNeeded(suite, IgniteCacheEntryProcessorNodeJoinTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, IgniteAtomicCacheEntryProcessorNodeJoinTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheNearTxForceKeyTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, CrossCacheTxRandomOperationsTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, CrossCacheTxNearEnabledRandomOperationsTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, IgniteDynamicCacheAndNodeStop.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, NearCacheSyncUpdateTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, CacheEnumOperationsSingleNodeTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, CacheEnumOperationsTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, IgniteCacheIncrementTxTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, IgniteCachePartitionedBackupNodeFailureRecoveryTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheVariableTopologySelfTest.class, ignoredTests);
 
-        suite.addTest(new TestSuite(IgniteCacheEntryProcessorNodeJoinTest.class));
-        suite.addTest(new TestSuite(IgniteAtomicCacheEntryProcessorNodeJoinTest.class));
-        suite.addTest(new TestSuite(GridCacheNearTxForceKeyTest.class));
-        suite.addTest(new TestSuite(CrossCacheTxRandomOperationsTest.class));
-        suite.addTest(new TestSuite(CrossCacheTxNearEnabledRandomOperationsTest.class));
-        suite.addTest(new TestSuite(IgniteDynamicCacheAndNodeStop.class));
-        suite.addTest(new TestSuite(CacheLockReleaseNodeLeaveTest.class));
-        suite.addTest(new TestSuite(NearCacheSyncUpdateTest.class));
-        suite.addTest(new TestSuite(CacheConfigurationLeakTest.class));
-        suite.addTest(new TestSuite(MemoryPolicyConfigValidationTest.class));
-        suite.addTest(new TestSuite(MemoryPolicyInitializationTest.class));
-        suite.addTest(new TestSuite(CacheGroupLocalConfigurationSelfTest.class));
-        suite.addTest(new TestSuite(CacheEnumOperationsSingleNodeTest.class));
-        suite.addTest(new TestSuite(CacheEnumOperationsTest.class));
-        suite.addTest(new TestSuite(IgniteCacheIncrementTxTest.class));
-        suite.addTest(new TestSuite(IgniteCachePartitionedBackupNodeFailureRecoveryTest.class));
+        GridTestUtils.addTestIfNeeded(suite, IgniteNoCustomEventsOnNodeStart.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, CacheExchangeMessageDuplicatedStateTest.class, ignoredTests);
 
-        suite.addTest(new TestSuite(IgniteNoCustomEventsOnNodeStart.class));
+        //GridTestUtils.addTestIfNeeded(suite,NearCacheMultithreadedUpdateTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite,NearCachePutAllMultinodeTest.class, ignoredTests);
 
-        suite.addTest(new TestSuite(CacheExchangeMessageDuplicatedStateTest.class));
-        suite.addTest(new TestSuite(CacheConcurrentReadThroughTest.class));
+        GridTestUtils.addTestIfNeeded(suite, IgniteOnePhaseCommitInvokeTest.class, ignoredTests);
 
-        suite.addTest(new TestSuite(GridNearCacheStoreUpdateTest.class));
-        //suite.addTest(new TestSuite(NearCacheMultithreadedUpdateTest.class));
-        //suite.addTest(new TestSuite(NearCachePutAllMultinodeTest.class));
+        GridTestUtils.addTestIfNeeded(suite, IgniteCacheNoSyncForGetTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite,IgniteCacheContainsKeyNearSelfTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite,IgniteCacheNearTxRollbackTest.class, ignoredTests);
 
-        suite.addTest(new TestSuite(IgniteOnePhaseCommitInvokeTest.class));
+        GridTestUtils.addTestIfNeeded(suite, IgniteOnePhaseCommitNearReadersTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, IgniteNearClientCacheCloseTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, IgniteClientCacheStartFailoverTest.class, ignoredTests);
 
-        suite.addTest(new TestSuite(IgniteCacheNoSyncForGetTest.class));
-        //suite.addTest(new TestSuite(IgniteCacheContainsKeyNearSelfTest.class));
-        //suite.addTest(new TestSuite(IgniteCacheNearTxRollbackTest.class));
+        GridTestUtils.addTestIfNeeded(suite, CacheOptimisticTransactionsWithFilterSingleServerTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, CacheOptimisticTransactionsWithFilterTest.class, ignoredTests);
 
-        suite.addTest(new TestSuite(IgniteOnePhaseCommitNearReadersTest.class));
-        suite.addTest(new TestSuite(IgniteNearClientCacheCloseTest.class));
-        suite.addTest(new TestSuite(IgniteClientCacheStartFailoverTest.class));
+        GridTestUtils.addTestIfNeeded(suite, NonAffinityCoordinatorDynamicStartStopTest.class, ignoredTests);
 
-        suite.addTest(new TestSuite(CacheOptimisticTransactionsWithFilterSingleServerTest.class));
-        suite.addTest(new TestSuite(CacheOptimisticTransactionsWithFilterTest.class));
+        GridTestUtils.addTestIfNeeded(suite, IgniteCacheClearDuringRebalanceTest.class, ignoredTests);
 
-        suite.addTest(new TestSuite(NonAffinityCoordinatorDynamicStartStopTest.class));
+        //GridTestUtils.addTestIfNeeded(suite,GridCacheColocatedDebugTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite, GridCacheDhtAtomicEvictionNearReadersSelfTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite,GridCacheDhtEntrySetSelfTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite,GridCacheDhtEvictionNearReadersSelfTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite,GridCacheDhtMultiBackupTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite,GridCacheDhtPreloadMessageCountTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite,GridCachePartitionedNearDisabledMetricsSelfTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite,IgniteCacheContainsKeyColocatedSelfTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite,IgniteCrossCacheTxNearEnabledSelfTest.class, ignoredTests);
+        //GridTestUtils.addTestIfNeeded(suite,IgniteTxConsistencyColocatedRestartSelfTest.class, ignoredTests);
 
-        suite.addTest(new TestSuite(IgniteCacheClearDuringRebalanceTest.class));
+        // Configuration validation
+        GridTestUtils.addTestIfNeeded(suite, CacheConfigurationLeakTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, MemoryPolicyConfigValidationTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, MemoryPolicyInitializationTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, CacheGroupLocalConfigurationSelfTest.class, ignoredTests);
 
-        suite.addTest(new TestSuite(CachePartitionStateTest.class));
+        // Affinity and collocation
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedAffinitySelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedProjectionAffinitySelfTest.class, ignoredTests);
 
-        suite.addTest(new TestSuite(CacheComparatorTest.class));
+        // Other tests.
+        GridTestUtils.addTestIfNeeded(suite, GridCacheNearJobExecutionSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheReplicatedJobExecutionTest.class, ignoredTests);
 
-        suite.addTest(new TestSuite(CachePartitionPartialCountersMapSelfTest.class));
+        //GridTestUtils.addTestIfNeeded(suite,RendezvousAffinityFunctionSelfTest.class), ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, RendezvousAffinityFunctionExcludeNeighborsSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, RendezvousAffinityFunctionFastPowerOfTwoHashSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, RendezvousAffinityFunctionStandardHashSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridCacheRendezvousAffinityClientSelfTest.class, ignoredTests);
 
-        suite.addTest(new TestSuite(IgniteReflectionFactorySelfTest.class));
+        GridTestUtils.addTestIfNeeded(suite, RendezvousAffinityFunctionBackupFilterSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, ClusterNodeAttributeAffinityBackupFilterSelfTest.class, ignoredTests);
 
-        //suite.addTest(new TestSuite(GridCacheColocatedDebugTest.class));
-        //suite.addTest(new TestSuite(GridCacheDhtAtomicEvictionNearReadersSelfTest.class));
-        //suite.addTest(new TestSuite(GridCacheDhtEntrySetSelfTest.class));
-        //suite.addTest(new TestSuite(GridCacheDhtEvictionNearReadersSelfTest.class));
-        //suite.addTest(new TestSuite(GridCacheDhtMultiBackupTest.class));
-        //suite.addTest(new TestSuite(GridCacheDhtPreloadMessageCountTest.class));
-        //suite.addTest(new TestSuite(GridCachePartitionedNearDisabledMetricsSelfTest.class));
-        //suite.addTest(new TestSuite(IgniteCacheContainsKeyColocatedSelfTest.class));
-        //suite.addTest(new TestSuite(IgniteCrossCacheTxNearEnabledSelfTest.class));
-        //suite.addTest(new TestSuite(IgniteTxConsistencyColocatedRestartSelfTest.class));
+        GridTestUtils.addTestIfNeeded(suite, CachePartitionStateTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, CacheComparatorTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, CachePartitionPartialCountersMapSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, IgniteReflectionFactorySelfTest.class, ignoredTests);
 
         return suite;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java
index c5384b9..7bb476f 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java
@@ -29,7 +29,6 @@ import org.apache.ignite.internal.processors.cache.ReplicatedTransactionalPessim
 import org.apache.ignite.internal.processors.cache.datastructures.IgniteExchangeLatchManagerCoordinatorFailTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheExchangeMergeTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheParallelStartTest;
-import org.apache.ignite.internal.processors.cache.distributed.CachePartitionStateTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheTryLockMultithreadedTest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCachePartitionEvictionDuringReadThroughSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCache150ClientsTest;
@@ -63,8 +62,6 @@ public class IgniteCacheTestSuite6 extends TestSuite {
     public static TestSuite suite() throws Exception {
         TestSuite suite = new TestSuite("IgniteCache Test Suite part 6");
 
-        suite.addTestSuite(CachePartitionStateTest.class);
-
         suite.addTestSuite(GridCachePartitionEvictionDuringReadThroughSelfTest.class);
         suite.addTestSuite(IgniteOptimisticTxSuspendResumeTest.class);
         suite.addTestSuite(IgniteOptimisticTxSuspendResumeMultiServerTest.class);


[45/50] [abbrv] ignite git commit: IGNITE-9700: Remove configurable values from mesos pom.xml. - Fixes #4838.

Posted by ag...@apache.org.
IGNITE-9700: Remove configurable values from mesos pom.xml. - Fixes #4838.

Signed-off-by: shroman <rs...@yahoo.com>


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

Branch: refs/heads/ignite-9720
Commit: 8e170d699ac081952c09bdba753e3a529916365c
Parents: c3fd4a9
Author: Roman Shtykh <rs...@yahoo.com>
Authored: Wed Nov 28 16:46:38 2018 +0900
Committer: shroman <rs...@yahoo.com>
Committed: Wed Nov 28 16:46:38 2018 +0900

----------------------------------------------------------------------
 modules/mesos/pom.xml                           | 48 --------------------
 .../ignite/mesos/resource/IgniteProvider.java   |  1 -
 2 files changed, 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8e170d69/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index 1841650..6234c81 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -36,9 +36,6 @@
 
     <properties>
         <mesos.version>1.5.0</mesos.version>
-        <ignite.version.url>https://ignite.apache.org/latest</ignite.version.url>
-        <ignite.path>/ignite/%s/apache-ignite-%s-bin.zip</ignite.path>
-        <apache.mirror.url>https://www.apache.org/dyn/closer.cgi?as_json=1</apache.mirror.url>
     </properties>
 
     <dependencies>
@@ -71,51 +68,6 @@
     <build>
         <plugins>
             <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-antrun-plugin</artifactId>
-                <version>1.7</version>
-                <executions>
-                    <execution>
-                        <id>update-versions</id>
-                        <goals>
-                            <goal>run</goal>
-                        </goals>
-                        <phase>generate-sources</phase>
-                        <configuration>
-                            <target>
-                                <echo message="Update download url in mesos module."/>
-                                <echo message="Latest version ${ignite.version.url}."/>
-
-                                <replaceregexp byline="true" encoding="UTF-8">
-                                    <regexp pattern="(.*IGNITE_LATEST_VERSION_URL = &quot;)(.*)(&quot;.*)"/>
-                                    <substitution expression="\1${ignite.version.url}\3"/>
-                                    <fileset dir="${basedir}/">
-                                        <include name="**/IgniteProvider.java"/>
-                                    </fileset>
-                                </replaceregexp>
-
-                                <replaceregexp byline="true" encoding="UTF-8">
-                                    <regexp pattern="(.*APACHE_MIRROR_URL = &quot;)(.*)(&quot;.*)"/>
-                                    <substitution expression="\1${apache.mirror.url}\3"/>
-                                    <fileset dir="${basedir}/">
-                                        <include name="**/IgniteProvider.java"/>
-                                    </fileset>
-                                </replaceregexp>
-
-                                <replaceregexp byline="true" encoding="UTF-8">
-                                    <regexp pattern="(.*IGNITE_PATH = &quot;)(.*)(&quot;.*)"/>
-                                    <substitution expression="\1${ignite.path}\3"/>
-                                    <fileset dir="${basedir}/">
-                                        <include name="**/IgniteProvider.java"/>
-                                    </fileset>
-                                </replaceregexp>
-                            </target>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-
-            <plugin>
                 <artifactId>maven-assembly-plugin</artifactId>
                 <version>2.4.1</version>
                 <configuration>

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e170d69/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 892a592..438f40d 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
@@ -47,7 +47,6 @@ public class IgniteProvider {
     /** Logger. */
     private static final Logger log = Logger.getLogger(IgniteProvider.class.getSimpleName());
 
-    // This constants are set by maven-ant-plugin.
     /** */
     private static final String DOWNLOAD_URL_PATTERN = "https://archive.apache.org/dist/ignite/%s/apache-ignite-%s-bin.zip";
 


[37/50] [abbrv] ignite git commit: IGNITE-10390 Fixed BPlusTree#isEmpty - Fixes #5486.

Posted by ag...@apache.org.
IGNITE-10390 Fixed BPlusTree#isEmpty - Fixes #5486.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-9720
Commit: 0b4282be26b4d042637ce57821e9a3448043d56d
Parents: 23de743
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Nov 27 19:31:51 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Nov 27 19:33:26 2018 +0300

----------------------------------------------------------------------
 .../persistence/GridCacheOffheapManager.java    |  4 +---
 .../cache/persistence/tree/BPlusTree.java       | 23 ++++++++++++--------
 .../processors/database/BPlusTreeSelfTest.java  | 23 ++++++++++++++++++++
 3 files changed, 38 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4282be/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
index 7d4a3ae..44ae6f1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
@@ -1463,7 +1463,6 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
          * @return Store delegate.
          * @throws IgniteCheckedException If failed.
          */
-        @SuppressWarnings("SizeReplaceableByIsEmpty")
         private CacheDataStore init0(boolean checkExists) throws IgniteCheckedException {
             CacheDataStore delegate0 = delegate;
 
@@ -1574,8 +1573,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
 
                     pendingTree = pendingTree0;
 
-                    // TODO IGNITE-10390 replace size() > 0 with isEmpty()
-                    if (!hasPendingEntries && pendingTree0.size() > 0)
+                    if (!hasPendingEntries && !pendingTree0.isEmpty())
                         hasPendingEntries = true;
 
                     int grpId = grp.groupId();

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4282be/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
index 7852008..476667a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
@@ -2109,21 +2109,26 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             long rootId, rootPage = acquirePage(rootId = treeMeta.rootId);
 
-            long rootAddr = readLock(rootId, rootPage);
+            try {
+                long rootAddr = readLock(rootId, rootPage);
 
-            if (rootAddr == 0) {
-                checkDestroyed();
+                if (rootAddr == 0) {
+                    checkDestroyed();
 
-                continue;
-            }
+                    continue;
+                }
 
-            try {
-                BPlusIO<L> io = io(rootAddr);
+                try {
+                    BPlusIO<L> io = io(rootAddr);
 
-                return io.getCount(rootAddr) == 0;
+                    return io.getCount(rootAddr) == 0;
+                }
+                finally {
+                    readUnlock(rootId, rootPage, rootAddr);
+                }
             }
             finally {
-                readUnlock(rootId, rootPage, rootAddr);
+                releasePage(rootId, rootPage);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b4282be/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
index fa72978..ae9aff8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
@@ -252,6 +252,29 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception if failed.
+     */
+    public void testIsEmpty() throws Exception {
+        TestTree tree = createTestTree(true);
+
+        assertTrue(tree.isEmpty());
+
+        for (long i = 1; i <= 500; i++) {
+            tree.put(i);
+
+            assertFalse(tree.isEmpty());
+        }
+
+        for (long i = 1; i <= 500; i++) {
+            assertFalse(tree.isEmpty());
+
+            tree.remove(i);
+        }
+
+        assertTrue(tree.isEmpty());
+    }
+
+    /**
      * @throws IgniteCheckedException If failed.
      */
     public void testFindWithClosure() throws IgniteCheckedException {


[07/50] [abbrv] ignite git commit: IGNITE-9828: Muted tests.

Posted by ag...@apache.org.
IGNITE-9828: Muted tests.


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

Branch: refs/heads/ignite-9720
Commit: ae5bfadf09254fd62416c887c1404713582a4571
Parents: 1dea0a4
Author: Igor Seliverstov <gv...@gmail.com>
Authored: Fri Nov 23 17:32:00 2018 +0300
Committer: Igor Seliverstov <gv...@gmail.com>
Committed: Fri Nov 23 17:32:22 2018 +0300

----------------------------------------------------------------------
 .../CacheContinuousQueryAsyncFailoverMvccTxSelfTest.java        | 5 +++++
 .../continuous/CacheContinuousQueryFailoverMvccTxSelfTest.java  | 5 +++++
 2 files changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ae5bfadf/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFailoverMvccTxSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFailoverMvccTxSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFailoverMvccTxSelfTest.java
index 334d219..7c15f38 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFailoverMvccTxSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFailoverMvccTxSelfTest.java
@@ -50,4 +50,9 @@ public class CacheContinuousQueryAsyncFailoverMvccTxSelfTest  extends CacheConti
     @Override public void testLeftPrimaryAndBackupNodes() throws Exception {
         fail("https://issues.apache.org/jira/browse/IGNITE-10047");
     }
+
+    /** {@inheritDoc} */
+    @Override public void testFailoverStartStopBackup() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-10391");
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ae5bfadf/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverMvccTxSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverMvccTxSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverMvccTxSelfTest.java
index 76d1689..2d578f2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverMvccTxSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverMvccTxSelfTest.java
@@ -45,4 +45,9 @@ public class CacheContinuousQueryFailoverMvccTxSelfTest extends CacheContinuousQ
     @Override public void testLeftPrimaryAndBackupNodes() throws Exception {
         fail("https://issues.apache.org/jira/browse/IGNITE-10047");
     }
+
+    /** {@inheritDoc} */
+    @Override public void testFailoverStartStopBackup() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-10391");
+    }
 }
\ No newline at end of file


[11/50] [abbrv] ignite git commit: IGNITE-9996: Performance drop final fix.

Posted by ag...@apache.org.
IGNITE-9996: Performance drop final fix.


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

Branch: refs/heads/ignite-9720
Commit: 475a0a71500dc4f638100f3ec0c7f8029db42f4a
Parents: 5c01c41
Author: Nikolay Izhikov <ni...@apache.org>
Authored: Fri Nov 23 21:22:12 2018 +0300
Committer: Nikolay Izhikov <ni...@apache.org>
Committed: Fri Nov 23 21:31:02 2018 +0300

----------------------------------------------------------------------
 .../cache/persistence/pagemem/PageMemoryImpl.java  |  6 +++---
 .../wal/serializer/RecordDataV1Serializer.java     | 17 ++++++++++++++---
 2 files changed, 17 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/475a0a71/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
index 6f7d2c5..b64b294 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
@@ -228,7 +228,7 @@ public class PageMemoryImpl implements PageMemoryEx {
     private final GridEncryptionManager encMgr;
 
     /** */
-    private final EncryptionSpi encSpi;
+    private final boolean encryptionDisabled;
 
     /** */
     private final IgniteLogger log;
@@ -324,7 +324,7 @@ public class PageMemoryImpl implements PageMemoryEx {
         storeMgr = ctx.pageStore();
         walMgr = ctx.wal();
         encMgr = ctx.kernalContext().encryption();
-        encSpi = ctx.gridConfig().getEncryptionSpi();
+        encryptionDisabled = ctx.gridConfig().getEncryptionSpi() instanceof  NoopEncryptionSpi;
 
         assert storeMgr != null;
         assert walMgr != null;
@@ -973,7 +973,7 @@ public class PageMemoryImpl implements PageMemoryEx {
 
     /** {@inheritDoc} */
     @Override public int realPageSize(int grpId) {
-        if ((encSpi instanceof NoopEncryptionSpi) || encMgr.groupKey(grpId) == null)
+        if (encryptionDisabled || encMgr.groupKey(grpId) == null)
             return pageSize();
 
         return encPageSize;

http://git-wip-us.apache.org/repos/asf/ignite/blob/475a0a71/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
index ab35f46..84c4074 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
@@ -146,6 +146,9 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
     private final GridEncryptionManager encMgr;
 
     /** */
+    private final boolean encryptionDisabled;
+
+    /** */
     private static final byte ENCRYPTED = 1;
 
     /** */
@@ -162,6 +165,8 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
         this.encSpi = cctx.gridConfig().getEncryptionSpi();
         this.encMgr = cctx.kernalContext().encryption();
 
+        encryptionDisabled = encSpi instanceof NoopEncryptionSpi;
+
         //This happen on offline WAL iteration(we don't have encryption keys available).
         if (encSpi != null)
             this.realPageSize = CU.encryptedPageSize(pageSize, encSpi);
@@ -227,7 +232,7 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
      * @return {@code True} if this record should be encrypted.
      */
     private boolean needEncryption(WALRecord rec) {
-        if (encSpi instanceof NoopEncryptionSpi)
+        if (encryptionDisabled)
             return false;
 
         if (!(rec instanceof WalRecordCacheGroupAware))
@@ -241,7 +246,7 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
      * @return {@code True} if this record should be encrypted.
      */
     private boolean needEncryption(int grpId) {
-        if (encSpi instanceof NoopEncryptionSpi)
+        if (encryptionDisabled)
             return false;
 
         GridEncryptionManager encMgr = cctx.kernalContext().encryption();
@@ -1923,6 +1928,9 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
      * @return Real record type.
      */
     RecordType recordType(WALRecord rec) {
+        if (encryptionDisabled)
+            return rec.type();
+
         if (needEncryption(rec))
             return ENCRYPTED_RECORD;
 
@@ -1937,6 +1945,9 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
      * @return {@code True} if this data record should be encrypted.
      */
     boolean isDataRecordEncrypted(DataRecord rec) {
+        if (encryptionDisabled)
+            return false;
+
         for (DataEntry e : rec.writeEntries()) {
             if (cctx.cacheContext(e.cacheId()) != null && needEncryption(cctx.cacheContext(e.cacheId()).groupId()))
                 return true;
@@ -2014,7 +2025,7 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
         for (DataEntry entry : dataRec.writeEntries()) {
             int clSz = entrySize(entry);
 
-            if (needEncryption(cctx.cacheContext(entry.cacheId()).groupId()))
+            if (!encryptionDisabled && needEncryption(cctx.cacheContext(entry.cacheId()).groupId()))
                 sz += encSpi.encryptedSize(clSz) + 1 /* encrypted flag */ + 4 /* groupId */ + 4 /* data size */;
             else {
                 sz += clSz;


[36/50] [abbrv] ignite git commit: IGNITE-7072: IgniteCache.replace(k, v, nv) fix for binary mode

Posted by ag...@apache.org.
IGNITE-7072: IgniteCache.replace(k, v, nv) fix for binary mode


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

Branch: refs/heads/ignite-9720
Commit: 23de74357483aa2979b54ea4e54254ef0d2f3637
Parents: 882b5a4
Author: Igor Sapego <is...@apache.org>
Authored: Tue Nov 27 19:26:26 2018 +0300
Committer: Igor Sapego <is...@apache.org>
Committed: Tue Nov 27 19:28:01 2018 +0300

----------------------------------------------------------------------
 .../cache/CacheEntryPredicateContainsValue.java |  4 ++
 .../GridCacheBinaryObjectsAbstractSelfTest.java | 58 ++++++++++++++++++++
 2 files changed, 62 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/23de7435/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateContainsValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateContainsValue.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateContainsValue.java
index 73a98b7..ad9861c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateContainsValue.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateContainsValue.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.CU;
@@ -60,6 +61,9 @@ public class CacheEntryPredicateContainsValue extends CacheEntryPredicateAdapter
 
         GridCacheContext cctx = e.context();
 
+        if (this.val instanceof BinaryObject && val instanceof BinaryObject)
+            return F.eq(val, this.val);
+
         Object thisVal = CU.value(this.val, cctx, false);
         Object cacheVal = CU.value(val, cctx, false);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/23de7435/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
index 00a3181..98e906f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
@@ -290,6 +290,64 @@ public abstract class GridCacheBinaryObjectsAbstractSelfTest extends GridCommonA
     /**
      * @throws Exception If failed.
      */
+    public void testReplace() throws Exception {
+        IgniteCache<Integer, TestObject> c = jcache(0);
+
+        for (int i = 0; i < ENTRY_CNT; i++)
+            c.put(i, new TestObject(i));
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            TestObject obj = c.get(i);
+
+            assertEquals(i, obj.val);
+        }
+
+        IgniteCache<Integer, BinaryObject> kpc = keepBinaryCache();
+
+        BinaryObjectBuilder bldr = grid(0).binary().builder(TestObject.class.getName());
+
+        bldr.setField("val", -42);
+
+        BinaryObject testObj = bldr.build();
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            BinaryObject po = kpc.get(i);
+
+            assertEquals(i, (int)po.field("val"));
+
+            assertTrue(kpc.replace(i, po, testObj));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRemove() throws Exception {
+        IgniteCache<Integer, TestObject> c = jcache(0);
+
+        for (int i = 0; i < ENTRY_CNT; i++)
+            c.put(i, new TestObject(i));
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            TestObject obj = c.get(i);
+
+            assertEquals(i, obj.val);
+        }
+
+        IgniteCache<Integer, BinaryObject> kpc = keepBinaryCache();
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            BinaryObject po = kpc.get(i);
+
+            assertEquals(i, (int)po.field("val"));
+
+            assertTrue(kpc.remove(i, po));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testIterator() throws Exception {
         IgniteCache<Integer, TestObject> c = jcache(0);
 


[15/50] [abbrv] ignite git commit: IGNITE-10330: Disk page compression. - Fixes #5200.

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java
index 4a12045..6176eeb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java
@@ -33,10 +33,12 @@ import org.apache.ignite.internal.util.GridStringBuilder;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.internal.util.GridUnsafe.bufferAddress;
+
 /**
  * Data pages IO.
  */
-public abstract class AbstractDataPageIO<T extends Storable> extends PageIO {
+public abstract class AbstractDataPageIO<T extends Storable> extends PageIO implements CompactablePageIO {
 
     /** */
     private static final int SHOW_ITEM = 0b0001;
@@ -228,7 +230,7 @@ public abstract class AbstractDataPageIO<T extends Storable> extends PageIO {
      * @param pageAddr Page address.
      * @return Free space.
      */
-    private int getRealFreeSpace(long pageAddr) {
+    public int getRealFreeSpace(long pageAddr) {
         return PageUtils.getShort(pageAddr, FREE_SPACE_OFF);
     }
 
@@ -822,9 +824,10 @@ public abstract class AbstractDataPageIO<T extends Storable> extends PageIO {
      * @param pageAddr Page address.
      * @param payload Payload.
      * @param pageSize Page size.
+     * @return Item ID.
      * @throws IgniteCheckedException If failed.
      */
-    public void addRow(
+    public int addRow(
         long pageAddr,
         byte[] payload,
         int pageSize
@@ -840,7 +843,7 @@ public abstract class AbstractDataPageIO<T extends Storable> extends PageIO {
 
         writeRowData(pageAddr, dataOff, payload);
 
-        addItem(pageAddr, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
+        return addItem(pageAddr, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
     }
 
     /**
@@ -1106,6 +1109,62 @@ public abstract class AbstractDataPageIO<T extends Storable> extends PageIO {
         return directCnt; // Previous directCnt will be our itemId.
     }
 
+    /** {@inheritDoc} */
+    @Override public void compactPage(ByteBuffer page, ByteBuffer out, int pageSize) {
+        // TODO May we compactDataEntries in-place and then copy compacted data to out?
+        copyPage(page, out, pageSize);
+
+        long pageAddr = bufferAddress(out);
+
+        int freeSpace = getRealFreeSpace(pageAddr);
+
+        if (freeSpace == 0)
+            return; // No garbage: nothing to compact here.
+
+        int directCnt = getDirectCount(pageAddr);
+
+        if (directCnt != 0) {
+            int firstOff = getFirstEntryOffset(pageAddr);
+
+            if (firstOff - freeSpace != getHeaderSizeWithItems(pageAddr, directCnt)) {
+                firstOff = compactDataEntries(pageAddr, directCnt, pageSize);
+                setFirstEntryOffset(pageAddr, firstOff, pageSize);
+            }
+
+            // Move all the data entries from page end to the page header to close the gap.
+            moveBytes(pageAddr, firstOff, pageSize - firstOff, -freeSpace, pageSize);
+        }
+
+        out.limit(pageSize - freeSpace); // Here we have only meaningful data of this page.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void restorePage(ByteBuffer page, int pageSize) {
+        assert page.isDirect();
+        assert page.position() == 0;
+        assert page.limit() <= pageSize;
+
+        long pageAddr = bufferAddress(page);
+
+        int freeSpace = getRealFreeSpace(pageAddr);
+
+        if (freeSpace != 0) {
+            int firstOff = getFirstEntryOffset(pageAddr);
+            int cnt = pageSize - firstOff;
+
+            if (cnt != 0) {
+                int off = page.limit() - cnt;
+
+                assert off > PageIO.COMMON_HEADER_END: off;
+                assert cnt > 0 : cnt;
+
+                moveBytes(pageAddr, off, cnt, freeSpace, pageSize);
+            }
+        }
+
+        page.limit(pageSize);
+    }
+
     /**
      * @param pageAddr Page address.
      * @param directCnt Direct items count.
@@ -1203,7 +1262,16 @@ public abstract class AbstractDataPageIO<T extends Storable> extends PageIO {
             entriesSize += entrySize;
         }
 
-        return pageSize - ITEMS_OFF - entriesSize - (directCnt + getIndirectCount(pageAddr)) * ITEM_SIZE;
+        return pageSize - entriesSize - getHeaderSizeWithItems(pageAddr, directCnt);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param directCnt Direct items count.
+     * @return Size of the page header including all items.
+     */
+    private int getHeaderSizeWithItems(long pageAddr, int directCnt) {
+        return ITEMS_OFF + (directCnt + getIndirectCount(pageAddr)) * ITEM_SIZE;
     }
 
     /**
@@ -1214,6 +1282,7 @@ public abstract class AbstractDataPageIO<T extends Storable> extends PageIO {
      * @param pageSize Page size.
      */
     private void moveBytes(long addr, int off, int cnt, int step, int pageSize) {
+        assert cnt >= 0: cnt;
         assert step != 0 : step;
         assert off + step >= 0;
         assert off + step + cnt <= pageSize : "[off=" + off + ", step=" + step + ", cnt=" + cnt +

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/BPlusIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/BPlusIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/BPlusIO.java
index 349e877..4c656eb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/BPlusIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/BPlusIO.java
@@ -17,16 +17,18 @@
 
 package org.apache.ignite.internal.processors.cache.persistence.tree.io;
 
+import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
 import org.apache.ignite.internal.util.GridStringBuilder;
+import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.lang.IgniteInClosure;
 
 /**
  * Abstract IO routines for B+Tree pages.
  */
-public abstract class BPlusIO<L> extends PageIO {
+public abstract class BPlusIO<L> extends PageIO implements CompactablePageIO {
     /** */
     private static final int CNT_OFF = COMMON_HEADER_END;
 
@@ -412,4 +414,32 @@ public abstract class BPlusIO<L> extends PageIO {
             .a(",\n\tremoveId=").appendHex(getRemoveId(addr))
             .a("\n]");
     }
+
+    /**
+     * @param pageAddr Page address.
+     * @return Offset after the last item.
+     */
+    public int getItemsEnd(long pageAddr) {
+        int cnt = getCount(pageAddr);
+        return offset(cnt);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void compactPage(ByteBuffer page, ByteBuffer out, int pageSize) {
+        copyPage(page, out, pageSize);
+
+        long pageAddr = GridUnsafe.bufferAddress(out);
+
+        // Just drop all the extra garbage at the end.
+        out.limit(getItemsEnd(pageAddr));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void restorePage(ByteBuffer compactPage, int pageSize) {
+        assert compactPage.isDirect();
+        assert compactPage.position() == 0;
+        assert compactPage.limit() <= pageSize;
+
+        compactPage.limit(pageSize); // Just add garbage to the end.
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/CompactablePageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/CompactablePageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/CompactablePageIO.java
new file mode 100644
index 0000000..775a1f8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/CompactablePageIO.java
@@ -0,0 +1,43 @@
+/*
+ * 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.internal.processors.cache.persistence.tree.io;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Page IO that supports compaction.
+ */
+public interface CompactablePageIO {
+    /**
+     * Compacts page contents to the output buffer.
+     * Implementation must not change contents, position and limit of the original page buffer.
+     *
+     * @param page Page buffer.
+     * @param out Output buffer.
+     * @param pageSize Page size.
+     */
+    void compactPage(ByteBuffer page, ByteBuffer out, int pageSize);
+
+    /**
+     * Restores the original page in place.
+     *
+     * @param compactPage Compact page.
+     * @param pageSize Page size.
+     */
+    void restorePage(ByteBuffer compactPage, int pageSize);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/DataPagePayload.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/DataPagePayload.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/DataPagePayload.java
index 49eed88..e58aad9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/DataPagePayload.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/DataPagePayload.java
@@ -17,6 +17,9 @@
 
 package org.apache.ignite.internal.processors.cache.persistence.tree.io;
 
+import org.apache.ignite.internal.pagemem.PageUtils;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
 /**
  *
  */
@@ -61,4 +64,17 @@ public class DataPagePayload {
     public long nextLink() {
         return nextLink;
     }
+
+    /**
+     * @param pageAddr Page address.
+     * @return Payload bytes.
+     */
+    public byte[] getBytes(long pageAddr) {
+        return PageUtils.getBytes(pageAddr, off, payloadSize);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(DataPagePayload.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
index ee61e25..85a1e8c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
@@ -83,6 +83,9 @@ import org.apache.ignite.internal.util.GridStringBuilder;
  */
 public abstract class PageIO {
     /** */
+    private static PageIO testIO;
+
+    /** */
     private static BPlusInnerIO<?> innerTestIO;
 
     /** */
@@ -131,16 +134,19 @@ public abstract class PageIO {
     public static final int ROTATED_ID_PART_OFF = PAGE_ID_OFF + 8;
 
     /** */
-    private static final int RESERVED_BYTE_OFF = ROTATED_ID_PART_OFF + 1;
+    private static final int COMPRESSION_TYPE_OFF = ROTATED_ID_PART_OFF + 1;
+
+    /** */
+    private static final int COMPRESSED_SIZE_OFF = COMPRESSION_TYPE_OFF + 1;
 
     /** */
-    private static final int RESERVED_SHORT_OFF = RESERVED_BYTE_OFF + 1;
+    private static final int COMPACTED_SIZE_OFF = COMPRESSED_SIZE_OFF + 2;
 
     /** */
-    private static final int RESERVED_INT_OFF = RESERVED_SHORT_OFF + 2;
+    private static final int RESERVED_SHORT_OFF = COMPACTED_SIZE_OFF + 2;
 
     /** */
-    private static final int RESERVED_2_OFF = RESERVED_INT_OFF + 4;
+    private static final int RESERVED_2_OFF = RESERVED_SHORT_OFF + 2;
 
     /** */
     private static final int RESERVED_3_OFF = RESERVED_2_OFF + 8;
@@ -382,6 +388,54 @@ public abstract class PageIO {
     }
 
     /**
+     * @param page Page buffer.
+     * @param compressType Compression type.
+     */
+    public static void setCompressionType(ByteBuffer page, byte compressType) {
+        page.put(COMPRESSION_TYPE_OFF, compressType);
+    }
+
+    /**
+     * @param page Page buffer.
+     * @return Compression type.
+     */
+    public static byte getCompressionType(ByteBuffer page) {
+        return page.get(COMPRESSION_TYPE_OFF);
+    }
+
+    /**
+     * @param page Page buffer.
+     * @param compressedSize Compressed size.
+     */
+    public static void setCompressedSize(ByteBuffer page, short compressedSize) {
+        page.putShort(COMPRESSED_SIZE_OFF, compressedSize);
+    }
+
+    /**
+     * @param page Page buffer.
+     * @return Compressed size.
+     */
+    public static short getCompressedSize(ByteBuffer page) {
+        return page.getShort(COMPRESSED_SIZE_OFF);
+    }
+
+    /**
+     * @param page Page buffer.
+     * @param compactedSize Compacted size.
+     */
+    public static void setCompactedSize(ByteBuffer page, short compactedSize) {
+        page.putShort(COMPACTED_SIZE_OFF, compactedSize);
+    }
+
+    /**
+     * @param page Page buffer.
+     * @return Compacted size.
+     */
+    public static short getCompactedSize(ByteBuffer page) {
+        return page.getShort(COMPACTED_SIZE_OFF);
+    }
+
+    /**
      * @param pageAddr Page address.
      * @return Checksum.
      */
@@ -487,6 +541,15 @@ public abstract class PageIO {
     }
 
     /**
+     * Registers IO for testing.
+     *
+     * @param io Page IO.
+     */
+    public static void registerTest(PageIO io) {
+        testIO = io;
+    }
+
+    /**
      * @return Type.
      */
     public final int getType() {
@@ -513,7 +576,8 @@ public abstract class PageIO {
         setPageId(pageAddr, pageId);
         setCrc(pageAddr, 0);
 
-        PageUtils.putLong(pageAddr, ROTATED_ID_PART_OFF, 0L); // 1 + reserved(1+2+4)
+        // rotated(1) + compress_type(1) + compressed_size(2) + compacted_size(2) + reserved(2)
+        PageUtils.putLong(pageAddr, ROTATED_ID_PART_OFF, 0L);
         PageUtils.putLong(pageAddr, RESERVED_2_OFF, 0L);
         PageUtils.putLong(pageAddr, RESERVED_3_OFF, 0L);
     }
@@ -536,6 +600,15 @@ public abstract class PageIO {
     }
 
     /**
+     * @param page Page.
+     * @return Page IO.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static <Q extends PageIO> Q getPageIO(ByteBuffer page) throws IgniteCheckedException {
+        return getPageIO(getType(page), getVersion(page));
+    }
+
+    /**
      * @param type IO Type.
      * @param ver IO Version.
      * @return Page IO.
@@ -572,6 +645,11 @@ public abstract class PageIO {
                 return (Q)SimpleDataPageIO.VERSIONS.forVersion(ver);
 
             default:
+                if (testIO != null) {
+                    if (testIO.type == type && testIO.ver == ver)
+                        return (Q)testIO;
+                }
+
                 return (Q)getBPlusIO(type, ver);
         }
     }
@@ -715,6 +793,21 @@ public abstract class PageIO {
     protected abstract void printPage(long addr, int pageSize, GridStringBuilder sb) throws IgniteCheckedException;
 
     /**
+     * @param page Page.
+     * @param out Output buffer.
+     * @param pageSize Page size.
+     */
+    protected final void copyPage(ByteBuffer page, ByteBuffer out, int pageSize) {
+        assert out.position() == 0;
+        assert pageSize <= out.remaining();
+        assert pageSize == page.remaining();
+
+        page.mark();
+        out.put(page).flip();
+        page.reset();
+    }
+
+    /**
      * @param addr Address.
      */
     public static String printPage(long addr, int pageSize) throws IgniteCheckedException {

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
index e70a027..03bac2b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
@@ -57,6 +57,7 @@ import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProces
 import org.apache.ignite.internal.processors.closure.GridClosureProcessor;
 import org.apache.ignite.internal.processors.cluster.ClusterProcessor;
 import org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor;
+import org.apache.ignite.internal.processors.compress.CompressionProcessor;
 import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor;
 import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor;
@@ -671,4 +672,9 @@ public class StandaloneGridKernalContext implements GridKernalContext {
     @NotNull @Override public Iterator<GridComponent> iterator() {
         return null;
     }
+
+    /** {@inheritDoc} */
+    @Override public CompressionProcessor compress() {
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/compress/CompressionProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/compress/CompressionProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/compress/CompressionProcessor.java
new file mode 100644
index 0000000..8b917b3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/compress/CompressionProcessor.java
@@ -0,0 +1,173 @@
+/*
+ * 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.internal.processors.compress;
+
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.DiskPageCompression;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteComponentType;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+
+/**
+ * Compression processor.
+ *
+ * @see IgniteComponentType#COMPRESSION
+ */
+public class CompressionProcessor extends GridProcessorAdapter {
+    /** */
+    public static final int LZ4_MIN_LEVEL = 0;
+
+    /** */
+    public static final int LZ4_MAX_LEVEL = 17;
+
+    /** */
+    public static final int LZ4_DEFAULT_LEVEL = 0;
+
+    /** */
+    public static final int ZSTD_MIN_LEVEL = -131072;
+
+    /** */
+    public static final int ZSTD_MAX_LEVEL = 22;
+
+    /** */
+    public static final int ZSTD_DEFAULT_LEVEL = 3;
+
+    /** */
+    protected static final byte UNCOMPRESSED_PAGE = 0;
+
+    /** */
+    protected static final byte COMPACTED_PAGE = 1;
+
+    /** */
+    protected static final byte ZSTD_COMPRESSED_PAGE = 2;
+
+    /** */
+    protected static final byte LZ4_COMPRESSED_PAGE = 3;
+
+    /** */
+    protected static final byte SNAPPY_COMPRESSED_PAGE = 4;
+
+    /**
+     * @param ctx Kernal context.
+     */
+    public CompressionProcessor(GridKernalContext ctx) {
+        super(ctx);
+    }
+
+    /**
+     * @param compression Compression algorithm.
+     * @return Default compression level.
+     */
+    public static int getDefaultCompressionLevel(DiskPageCompression compression) {
+        switch (compression) {
+            case ZSTD:
+                return ZSTD_DEFAULT_LEVEL;
+
+            case LZ4:
+                return LZ4_DEFAULT_LEVEL;
+
+            case SNAPPY:
+            case SKIP_GARBAGE:
+                return 0;
+        }
+
+        throw new IllegalArgumentException("Compression: " + compression);
+    }
+
+    /**
+     * @param compressLevel Compression level.
+     * @param compression Compression algorithm.
+     * @return Compression level.
+     */
+    public static int checkCompressionLevelBounds(int compressLevel, DiskPageCompression compression) {
+        switch (compression) {
+            case ZSTD:
+                checkCompressionLevelBounds(compressLevel, ZSTD_MIN_LEVEL, ZSTD_MAX_LEVEL);
+                break;
+
+            case LZ4:
+                checkCompressionLevelBounds(compressLevel, LZ4_MIN_LEVEL, LZ4_MAX_LEVEL);
+                break;
+
+            default:
+                throw new IllegalArgumentException("Compression level for " + compression + " is not supported.");
+        }
+
+        return compressLevel;
+    }
+
+    /**
+     * @param compressLevel Compression level.
+     * @param min Min level.
+     * @param max Max level.
+     */
+    private static void checkCompressionLevelBounds(int compressLevel, int min, int max) {
+        if (compressLevel < min  || compressLevel > max) {
+            throw new IllegalArgumentException("Compression level for LZ4 must be between " + min +
+                " and " + max + ".");
+        }
+    }
+
+    /**
+     * @throws IgniteCheckedException Always.
+     */
+    private static <T> T fail() throws IgniteCheckedException {
+        throw new IgniteCheckedException("Make sure that ignite-compress module is in classpath.");
+    }
+
+    /**
+     * @param storagePath Storage path.
+     * @param pageSize Page size.
+     * @throws IgniteCheckedException If compression is not supported.
+     */
+    public void checkPageCompressionSupported(Path storagePath, int pageSize) throws IgniteCheckedException {
+        fail();
+    }
+
+    /**
+     * @param page Page buffer.
+     * @param pageSize Page size.
+     * @param storeBlockSize Store block size.
+     * @param compression Compression algorithm.
+     * @param compressLevel Compression level.
+     * @return Possibly compressed buffer.
+     * @throws IgniteCheckedException If failed.
+     */
+    public ByteBuffer compressPage(
+        ByteBuffer page,
+        int pageSize,
+        int storeBlockSize,
+        DiskPageCompression compression,
+        int compressLevel
+    ) throws IgniteCheckedException {
+        return fail();
+    }
+
+    /**
+     * @param page Possibly compressed page buffer.
+     * @param pageSize Page size.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void decompressPage(ByteBuffer page, int pageSize) throws IgniteCheckedException {
+        if (PageIO.getCompressionType(page) != UNCOMPRESSED_PAGE)
+            fail();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/compress/FileSystemUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/compress/FileSystemUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/compress/FileSystemUtils.java
new file mode 100644
index 0000000..1877640
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/compress/FileSystemUtils.java
@@ -0,0 +1,129 @@
+/*
+ * 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.internal.processors.compress;
+
+import java.nio.file.Path;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.IgniteComponentType;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Native file system API.
+ */
+public final class FileSystemUtils {
+    /** */
+    private static final String NATIVE_FS_LINUX_CLASS =
+        "org.apache.ignite.internal.processors.compress.NativeFileSystemLinux";
+
+    /** */
+    private static final NativeFileSystem fs;
+
+    /** */
+    private static volatile Throwable err;
+
+    /** */
+    static {
+        NativeFileSystem x = null;
+
+        try {
+            if (IgniteComponentType.COMPRESSION.inClassPath()) {
+                if (U.isLinux())
+                    x = U.newInstance(NATIVE_FS_LINUX_CLASS);
+            }
+        }
+        catch (Throwable e) {
+            err = e;
+        }
+
+        fs = x;
+    }
+
+    /**
+     */
+    public static void checkSupported() {
+        Throwable e = err;
+
+        if (e != null || fs == null)
+            throw new IgniteException("Native file system API is not supported on " + U.osString(), e);
+    }
+
+    /**
+     * @param path File system path.
+     * @return File system block size or negative value if not supported.
+     */
+    public static int getFileSystemBlockSize(Path path) {
+        return fs == null ? -1 : fs.getFileSystemBlockSize(path);
+    }
+
+    /**
+     * @param fd Native file descriptor.
+     * @return File system block size or negative value if not supported.
+     */
+    public static int getFileSystemBlockSize(int fd) {
+        return fs == null ? -1 : fs.getFileSystemBlockSize(fd);
+    }
+
+    /**
+     * !!! Use with caution. May produce unexpected results.
+     *
+     * Known to work correctly on Linux EXT4 and Btrfs,
+     * while on XSF it returns meaningful result only after
+     * file reopening.
+     *
+     * @param fd Native file descriptor.
+     * @return Approximate system dependent size of the sparse file or negative
+     *          value if not supported.
+     */
+    public static long getSparseFileSize(int fd) {
+        return fs == null ? -1 : fs.getSparseFileSize(fd);
+    }
+
+    /**
+     * @param fd Native file descriptor.
+     * @param off Offset of the hole.
+     * @param len Length of the hole.
+     * @param fsBlockSize File system block size.
+     * @return Actual punched hole size.
+     */
+    public static long punchHole(int fd, long off, long len, int fsBlockSize) {
+        assert off >= 0;
+        assert len > 0;
+
+        checkSupported();
+
+        if (len < fsBlockSize)
+            return 0;
+
+        // TODO maybe optimize for power of 2
+        if (off % fsBlockSize != 0) {
+            long end = off + len;
+            off = (off / fsBlockSize + 1) * fsBlockSize;
+            len = end - off;
+
+            if (len <= 0)
+                return 0;
+        }
+
+        len = len / fsBlockSize * fsBlockSize;
+
+        if (len > 0)
+            fs.punchHole(fd, off, len);
+
+        return len;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/processors/compress/NativeFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/compress/NativeFileSystem.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/compress/NativeFileSystem.java
new file mode 100644
index 0000000..673d1bc
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/compress/NativeFileSystem.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.compress;
+
+import java.nio.file.Path;
+
+/**
+ * Native file system API.
+ */
+public interface NativeFileSystem {
+    /**
+     * @param path Path.
+     * @return File system block size in bytes.
+     */
+    int getFileSystemBlockSize(Path path);
+
+    /**
+     * @param fd Native file descriptor.
+     * @return File system block size in bytes.
+     */
+    int getFileSystemBlockSize(int fd);
+
+    /**
+     * @param fd Native file descriptor.
+     * @param off Offset of the hole.
+     * @param len Length of the hole.
+     */
+    void punchHole(int fd, long off, long len);
+
+    /**
+     * @param fd Native file descriptor.
+     * @return Approximate system dependent size of the sparse file.
+     */
+    long getSparseFileSize(int fd);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
index 2cc0ae3..6b8e2b3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
@@ -1339,6 +1339,7 @@ public abstract class GridUnsafe {
      * @return Buffer memory address.
      */
     public static long bufferAddress(ByteBuffer buf) {
+        assert buf.isDirect();
         return UNSAFE.getLong(buf, DIRECT_BUF_ADDR_OFF);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index 2d6b584..946378d 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -3933,15 +3933,23 @@ public abstract class IgniteUtils {
      * @return Hex string.
      */
     public static String byteArray2HexString(byte[] arr) {
-        SB sb = new SB(arr.length << 1);
+        StringBuilder sb = new StringBuilder(arr.length << 1);
 
         for (byte b : arr)
-            sb.a(Integer.toHexString(MASK & b >>> 4)).a(Integer.toHexString(MASK & b));
+            addByteAsHex(sb, b);
 
         return sb.toString().toUpperCase();
     }
 
     /**
+     * @param sb String builder.
+     * @param b Byte to add in hexadecimal format.
+     */
+    private static void addByteAsHex(StringBuilder sb, byte b) {
+        sb.append(Integer.toHexString(MASK & b >>> 4)).append(Integer.toHexString(MASK & b));
+    }
+
+    /**
      * Checks for containment of the value in the array.
      * Both array cells and value may be {@code null}. Two {@code null}s are considered equal.
      *
@@ -10552,12 +10560,10 @@ public abstract class IgniteUtils {
      * @return hex representation of memory region
      */
     public static String toHexString(long addr, int len) {
-        assert (len & 0b111) == 0 && len > 0;
-
         StringBuilder sb = new StringBuilder(len * 2);
 
-        for (int i = 0; i < len; i += 8)
-            sb.append(U.hexLong(GridUnsafe.getLong(addr + i)));
+        for (int i = 0; i < len; i++) // Can not use getLong because on little-endian it produces bs.
+            addByteAsHex(sb, GridUnsafe.getByte(addr + i));
 
         return sb.toString();
     }
@@ -10568,12 +10574,10 @@ public abstract class IgniteUtils {
      * @return hex representation of memory region
      */
     public static String toHexString(ByteBuffer buf) {
-        assert (buf.capacity() & 0b111) == 0;
-
         StringBuilder sb = new StringBuilder(buf.capacity() * 2);
 
-        for (int i = 0; i < buf.capacity(); i += 8)
-            sb.append(U.hexLong(buf.getLong(i)));
+        for (int i = 0; i < buf.capacity(); i++)
+            addByteAsHex(sb, buf.get(i)); // Can not use getLong because on little-endian it produces bs.
 
         return sb.toString();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/mxbean/CacheGroupMetricsMXBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/CacheGroupMetricsMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/CacheGroupMetricsMXBean.java
index 2612a41..cf8aa76 100644
--- a/modules/core/src/main/java/org/apache/ignite/mxbean/CacheGroupMetricsMXBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/mxbean/CacheGroupMetricsMXBean.java
@@ -179,4 +179,16 @@ public interface CacheGroupMetricsMXBean {
      */
     @MXBeanDescription("Total size of memory allocated for group, in bytes.")
     public long getTotalAllocatedSize();
+
+    /**
+     * Storage space allocated for group, in bytes.
+     */
+    @MXBeanDescription("Storage space allocated for group, in bytes.")
+    public long getStorageSize();
+
+    /**
+     * Storage space allocated for group adjusted for possible sparsity, in bytes.
+     */
+    @MXBeanDescription("Storage space allocated for group adjusted for possible sparsity, in bytes.")
+    public long getSparseStorageSize();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/main/java/org/apache/ignite/mxbean/DataStorageMetricsMXBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/DataStorageMetricsMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/DataStorageMetricsMXBean.java
index 2069099..4689f15 100644
--- a/modules/core/src/main/java/org/apache/ignite/mxbean/DataStorageMetricsMXBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/mxbean/DataStorageMetricsMXBean.java
@@ -174,4 +174,12 @@ public interface DataStorageMetricsMXBean extends DataStorageMetrics {
         "Number of subintervals to set."
     )
     public void subIntervals(int subInts);
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Storage space allocated, in bytes.")
+    @Override long getStorageSize();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Storage space allocated adjusted for possible sparsity, in bytes.")
+    @Override long getSparseStorageSize();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
index 36d8b41..ed6eb86 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
@@ -120,7 +120,6 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
         }
 
         DataStorageConfiguration memCfg = new DataStorageConfiguration();
-        memCfg.setPageSize(4 * 1024);
         memCfg.setDefaultDataRegionConfiguration(new DataRegionConfiguration()
             .setMaxSize(150L * 1024 * 1024)
             .setPersistenceEnabled(persistenceEnabled()));
@@ -1330,6 +1329,7 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
         ccfgs[4] = cacheConfiguration(CACHE_NAME_PREFIX + 4, TRANSACTIONAL);
 
         ccfgs[4].setDataRegionName(NO_PERSISTENCE_REGION);
+        ccfgs[4].setDiskPageCompression(null);
 
         return ccfgs;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/DummyPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/DummyPageIO.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/DummyPageIO.java
index 1b36ac1..c402ad7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/DummyPageIO.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/DummyPageIO.java
@@ -17,25 +17,40 @@
 
 package org.apache.ignite.internal.processors.cache.persistence;
 
+import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.CompactablePageIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.util.GridStringBuilder;
 
 /**
  * Dummy PageIO implementation. For test purposes only.
  */
-public class DummyPageIO extends PageIO {
+public class DummyPageIO extends PageIO implements CompactablePageIO {
     /** */
     public DummyPageIO() {
         super(2 * Short.MAX_VALUE, 1);
+
+        PageIO.registerTest(this);
     }
 
     /** {@inheritDoc} */
-    @Override
-    protected void printPage(long addr, int pageSize, GridStringBuilder sb) throws IgniteCheckedException {
+    @Override protected void printPage(long addr, int pageSize, GridStringBuilder sb) throws IgniteCheckedException {
         sb.a("DummyPageIO [\n");
         sb.a("addr=").a(addr).a(", ");
         sb.a("pageSize=").a(addr);
         sb.a("\n]");
     }
+
+    /** {@inheritDoc} */
+    @Override public void compactPage(ByteBuffer page, ByteBuffer out, int pageSize) {
+        copyPage(page, out, pageSize);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void restorePage(ByteBuffer p, int pageSize) {
+        assert p.isDirect();
+        assert p.position() == 0;
+        assert p.limit() == pageSize;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteDataStorageMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteDataStorageMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteDataStorageMetricsSelfTest.java
index 4db1de9..0f5aef9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteDataStorageMetricsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteDataStorageMetricsSelfTest.java
@@ -55,6 +55,9 @@ public class IgniteDataStorageMetricsSelfTest extends GridCommonAbstractTest {
     /** */
     private static final String GROUP1 = "grp1";
 
+    /** */
+    private static final String NO_PERSISTENCE = "no-persistence";
+
     /** {@inheritDoc} */
     @Override protected void afterTestsStopped() throws Exception {
         cleanPersistenceDir();
@@ -73,19 +76,20 @@ public class IgniteDataStorageMetricsSelfTest extends GridCommonAbstractTest {
 
         cfg.setConsistentId(gridName);
 
+        long maxRegionSize = 20L * 1024 * 1024;
+
         DataStorageConfiguration memCfg = new DataStorageConfiguration()
             .setDefaultDataRegionConfiguration(new DataRegionConfiguration()
-                .setMaxSize(10L * 1024 * 1024)
+                .setMaxSize(maxRegionSize)
                 .setPersistenceEnabled(true)
                 .setMetricsEnabled(true)
                 .setName("dflt-plc"))
             .setDataRegionConfigurations(new DataRegionConfiguration()
-                .setMaxSize(10L * 1024 * 1024)
+                .setMaxSize(maxRegionSize)
                 .setPersistenceEnabled(false)
                 .setMetricsEnabled(true)
-                .setName("no-persistence"))
+                .setName(NO_PERSISTENCE))
             .setWalMode(WALMode.LOG_ONLY)
-            .setPageSize(4 * 1024)
             .setMetricsEnabled(true);
 
         cfg.setDataStorageConfiguration(memCfg);
@@ -95,7 +99,7 @@ public class IgniteDataStorageMetricsSelfTest extends GridCommonAbstractTest {
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
 
         cfg.setCacheConfiguration(cacheConfiguration(GROUP1, "cache", PARTITIONED, ATOMIC, 1, null),
-            cacheConfiguration(null, "cache-np", PARTITIONED, ATOMIC, 1, "no-persistence"));
+            cacheConfiguration(null, "cache-np", PARTITIONED, ATOMIC, 1, NO_PERSISTENCE));
 
         return cfg;
     }
@@ -135,6 +139,9 @@ public class IgniteDataStorageMetricsSelfTest extends GridCommonAbstractTest {
         ccfg.setWriteSynchronizationMode(FULL_SYNC);
         ccfg.setDataRegionName(dataRegName);
 
+        if (NO_PERSISTENCE.equals(dataRegName))
+            ccfg.setDiskPageCompression(null);
+
         return ccfg;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDynamicCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDynamicCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDynamicCacheTest.java
index 713d4cc..be3ed0b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDynamicCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDynamicCacheTest.java
@@ -45,8 +45,7 @@ public class IgnitePdsDynamicCacheTest extends IgniteDbDynamicCacheSelfTest {
         DataStorageConfiguration memCfg = new DataStorageConfiguration()
             .setDefaultDataRegionConfiguration(
                 new DataRegionConfiguration().setMaxSize(200L * 1024 * 1024).setPersistenceEnabled(true))
-            .setWalMode(WALMode.LOG_ONLY)
-            .setPageSize(4 * 1024);
+            .setWalMode(WALMode.LOG_ONLY);
 
         cfg.setDataStorageConfiguration(memCfg);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsExchangeDuringCheckpointTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsExchangeDuringCheckpointTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsExchangeDuringCheckpointTest.java
index ecc6e02..d6d5c4f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsExchangeDuringCheckpointTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsExchangeDuringCheckpointTest.java
@@ -105,6 +105,7 @@ public class IgnitePdsExchangeDuringCheckpointTest extends GridCommonAbstractTes
 
         CacheConfiguration ccfgNp = new CacheConfiguration("nonPersistentCache");
         ccfgNp.setDataRegionName(NO_PERSISTENCE_REGION);
+        ccfgNp.setDiskPageCompression(null);
 
         ccfg.setAffinity(new RendezvousAffinityFunction(false, 4096));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPageSizesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPageSizesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPageSizesTest.java
index 353bc50..d907239 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPageSizesTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPageSizesTest.java
@@ -21,16 +21,20 @@ import java.util.Random;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ThreadLocalRandom;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.DiskPageCompression;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_DEFAULT_DISK_PAGE_COMPRESSION;
+
 /**
  *
  */
@@ -112,6 +116,10 @@ public class IgnitePdsPageSizesTest extends GridCommonAbstractTest {
      * @throws Exception if failed.
      */
     private void checkPageSize(int pageSize) throws Exception {
+        if (pageSize <= 4 * 1024 &&
+            IgniteSystemProperties.getEnum(DiskPageCompression.class, IGNITE_DEFAULT_DISK_PAGE_COMPRESSION) != null)
+            return; // Small pages do not work with compression.
+
         this.pageSize = pageSize;
 
         IgniteEx ignite = startGrid(0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java
index 48b60d4..56426f3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java
@@ -566,6 +566,21 @@ public class LocalWalModeChangeDuringRebalancingSelfTest extends GridCommonAbstr
         }
 
         /** {@inheritDoc} */
+        @Override public int getFileSystemBlockSize() {
+            return delegate.getFileSystemBlockSize();
+        }
+
+        /** {@inheritDoc} */
+        @Override public long getSparseSize() {
+            return delegate.getSparseSize();
+        }
+
+        /** {@inheritDoc} */
+        @Override public int punchHole(long position, int len) {
+            return delegate.punchHole(position, len);
+        }
+
+        /** {@inheritDoc} */
         @Override public long position() throws IOException {
             return delegate.position();
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsCacheRestoreTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsCacheRestoreTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsCacheRestoreTest.java
index b8cb047..4787143 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsCacheRestoreTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsCacheRestoreTest.java
@@ -57,14 +57,15 @@ public class IgnitePdsCacheRestoreTest extends GridCommonAbstractTest {
             ccfgs = null;
         }
 
+        long regionMaxSize = 20L * 1024 * 1024;
+
         DataStorageConfiguration memCfg = new DataStorageConfiguration()
             .setDefaultDataRegionConfiguration(
-                new DataRegionConfiguration().setMaxSize(10L * 1024 * 1024).setPersistenceEnabled(true))
-            .setPageSize(4 * 1024)
+                new DataRegionConfiguration().setMaxSize(regionMaxSize).setPersistenceEnabled(true))
             .setWalMode(WALMode.LOG_ONLY);
 
         memCfg.setDataRegionConfigurations(new DataRegionConfiguration()
-            .setMaxSize(10L * 1024 * 1024)
+            .setMaxSize(regionMaxSize)
             .setName(NO_PERSISTENCE_REGION)
             .setPersistenceEnabled(false));
 
@@ -210,6 +211,7 @@ public class IgnitePdsCacheRestoreTest extends GridCommonAbstractTest {
         ccfgs[2] = cacheConfiguration("c3");
 
         ccfgs[2].setDataRegionName(NO_PERSISTENCE_REGION);
+        ccfgs[2].setDiskPageCompression(null);
 
         return ccfgs;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java
index c05f65c..0154c14 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java
@@ -60,7 +60,7 @@ public class IgnitePdsDataRegionMetricsTest extends GridCommonAbstractTest {
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
     /** */
-    private static final long INIT_REGION_SIZE = 10 << 20;
+    private static final long INIT_REGION_SIZE = 20 << 20;
 
     /** */
     private static final long MAX_REGION_SIZE = INIT_REGION_SIZE * 10;
@@ -312,7 +312,13 @@ public class IgnitePdsDataRegionMetricsTest extends GridCommonAbstractTest {
 
                 FilePageStore store = (FilePageStore)pageStoreMgr.getStore(CU.cacheId(cacheName), partId(file));
 
-                totalPersistenceSize += path.toFile().length() - store.headerSize();
+                int pageSize = store.getPageSize();
+                long storeSize = path.toFile().length() - store.headerSize();
+
+                if (storeSize % pageSize != 0)
+                    storeSize = (storeSize / pageSize + 1) * pageSize; // Adjust for possible page compression.
+
+                totalPersistenceSize += storeSize;
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsReserveWalSegmentsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsReserveWalSegmentsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsReserveWalSegmentsTest.java
index 5593e44..cfbb8d8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsReserveWalSegmentsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsReserveWalSegmentsTest.java
@@ -61,8 +61,6 @@ public class IgnitePdsReserveWalSegmentsTest extends GridCommonAbstractTest {
 
         DataStorageConfiguration dbCfg = new DataStorageConfiguration();
 
-        dbCfg.setPageSize(4 * 1024);
-
         cfg.setDataStorageConfiguration(dbCfg);
 
         dbCfg.setWalSegmentSize(1024 * 1024)

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/IgniteCheckpointDirtyPagesForLowLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/IgniteCheckpointDirtyPagesForLowLoadTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/IgniteCheckpointDirtyPagesForLowLoadTest.java
index 8baa1c3..c9174ba 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/IgniteCheckpointDirtyPagesForLowLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/IgniteCheckpointDirtyPagesForLowLoadTest.java
@@ -68,7 +68,6 @@ public class IgniteCheckpointDirtyPagesForLowLoadTest extends GridCommonAbstract
         cfg.setCacheConfiguration(ccfgs.toArray(new CacheConfiguration[ccfgs.size()]));
 
         DataStorageConfiguration dsCfg = new DataStorageConfiguration();
-        dsCfg.setPageSize(1024); //smaller page to reduce overhead to short values
         dsCfg.setDefaultDataRegionConfiguration(
             new DataRegionConfiguration()
                 .setPersistenceEnabled(true)

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/DefaultPageSizeBackwardsCompatibilityTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/DefaultPageSizeBackwardsCompatibilityTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/DefaultPageSizeBackwardsCompatibilityTest.java
index 29e113c..70d003d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/DefaultPageSizeBackwardsCompatibilityTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/DefaultPageSizeBackwardsCompatibilityTest.java
@@ -38,7 +38,7 @@ public class DefaultPageSizeBackwardsCompatibilityTest extends GridCommonAbstrac
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
     /** Client mode. */
-    private boolean set2kPageSize = true;
+    private boolean set16kPageSize = true;
 
     /** Entries count. */
     public static final int ENTRIES_COUNT = 300;
@@ -55,8 +55,10 @@ public class DefaultPageSizeBackwardsCompatibilityTest extends GridCommonAbstrac
 
         DataStorageConfiguration memCfg = new DataStorageConfiguration();
 
-        if (set2kPageSize)
-            memCfg.setPageSize(2048);
+        if (set16kPageSize)
+            memCfg.setPageSize(16 * 1024);
+        else
+            memCfg.setPageSize(0); // Enforce default.
 
         DataRegionConfiguration memPlcCfg = new DataRegionConfiguration();
         memPlcCfg.setMaxSize(100L * 1000 * 1000);
@@ -64,7 +66,7 @@ public class DefaultPageSizeBackwardsCompatibilityTest extends GridCommonAbstrac
         memPlcCfg.setPersistenceEnabled(true);
 
         memCfg.setDefaultDataRegionConfiguration(memPlcCfg);
-        memCfg.setCheckpointFrequency(3_000);
+        memCfg.setCheckpointFrequency(500);
 
         cfg.setDataStorageConfiguration(memCfg);
 
@@ -75,6 +77,9 @@ public class DefaultPageSizeBackwardsCompatibilityTest extends GridCommonAbstrac
         ccfg1.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
         ccfg1.setAffinity(new RendezvousAffinityFunction(false, 32));
 
+        if (!set16kPageSize)
+            ccfg1.setDiskPageCompression(null);
+
         cfg.setCacheConfiguration(ccfg1);
 
         cfg.setConsistentId(gridName);
@@ -99,7 +104,7 @@ public class DefaultPageSizeBackwardsCompatibilityTest extends GridCommonAbstrac
     /**
      * @throws Exception If failed.
      */
-    public void testStartFrom2kDefaultStore() throws Exception {
+    public void testStartFrom16kDefaultStore() throws Exception {
         startGrids(2);
 
         Ignite ig = ignite(0);
@@ -113,11 +118,11 @@ public class DefaultPageSizeBackwardsCompatibilityTest extends GridCommonAbstrac
         for (int i = 0; i < ENTRIES_COUNT; i++)
             cache.put(i, i);
 
-        Thread.sleep(5_000); // Await for checkpoint to happen.
+        Thread.sleep(1500); // Await for checkpoint to happen.
 
         stopAllGrids();
 
-        set2kPageSize = false;
+        set16kPageSize = false;
 
         startGrids(2);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java
index 5bf7e7f..14525f0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionTest.java
@@ -42,8 +42,6 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
-import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_PAGE_SIZE;
-
 /**
  *
  */
@@ -83,7 +81,7 @@ public class WalCompactionTest extends GridCommonAbstractTest {
 
         CacheConfiguration ccfg = new CacheConfiguration();
 
-        ccfg.setName("cache");
+        ccfg.setName(CACHE_NAME);
         ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
         ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
         ccfg.setAffinity(new RendezvousAffinityFunction(false, 16));
@@ -139,7 +137,9 @@ public class WalCompactionTest extends GridCommonAbstractTest {
         IgniteEx ig = (IgniteEx)startGrids(3);
         ig.cluster().active(true);
 
-        IgniteCache<Integer, byte[]> cache = ig.cache("cache");
+        IgniteCache<Integer, byte[]> cache = ig.cache(CACHE_NAME);
+
+        final int pageSize = ig.cachex(CACHE_NAME).context().dataRegion().pageMemory().pageSize();
 
         for (int i = 0; i < ENTRIES; i++) { // At least 20MB of raw data in total.
             final byte[] val = new byte[20000];
@@ -150,9 +150,9 @@ public class WalCompactionTest extends GridCommonAbstractTest {
         }
 
         // Spam WAL to move all data records to compressible WAL zone.
-        for (int i = 0; i < WAL_SEGMENT_SIZE / DFLT_PAGE_SIZE * 2; i++) {
-            ig.context().cache().context().wal().log(new PageSnapshot(new FullPageId(-1, -1), new byte[DFLT_PAGE_SIZE],
-                DFLT_PAGE_SIZE));
+        for (int i = 0; i < WAL_SEGMENT_SIZE / pageSize * 2; i++) {
+            ig.context().cache().context().wal().log(new PageSnapshot(new FullPageId(-1, -1), new byte[pageSize],
+                pageSize));
         }
 
         // WAL archive segment is allowed to be compressed when it's at least one checkpoint away from current WAL head.
@@ -262,7 +262,7 @@ public class WalCompactionTest extends GridCommonAbstractTest {
         IgniteEx ig = startGrid(0);
         ig.cluster().active(true);
 
-        IgniteCache<Integer, byte[]> cache = ig.cache("cache");
+        IgniteCache<Integer, byte[]> cache = ig.cache(CACHE_NAME);
 
         for (int i = 0; i < 2500; i++) { // At least 50MB of raw data in total.
             final byte[] val = new byte[20000];
@@ -325,7 +325,9 @@ public class WalCompactionTest extends GridCommonAbstractTest {
         IgniteEx ig = (IgniteEx)startGrids(3);
         ig.cluster().active(true);
 
-        IgniteCache<Integer, byte[]> cache = ig.cache("cache");
+        IgniteCache<Integer, byte[]> cache = ig.cache(CACHE_NAME);
+
+        final int pageSize = ig.cachex(CACHE_NAME).context().dataRegion().pageMemory().pageSize();
 
         for (int i = 0; i < 100; i++) {
             final byte[] val = new byte[20000];
@@ -364,9 +366,9 @@ public class WalCompactionTest extends GridCommonAbstractTest {
         }
 
         // Spam WAL to move all data records to compressible WAL zone.
-        for (int i = 0; i < WAL_SEGMENT_SIZE / DFLT_PAGE_SIZE * 2; i++) {
-            ig.context().cache().context().wal().log(new PageSnapshot(new FullPageId(-1, -1), new byte[DFLT_PAGE_SIZE],
-                DFLT_PAGE_SIZE));
+        for (int i = 0; i < WAL_SEGMENT_SIZE / pageSize * 2; i++) {
+            ig.context().cache().context().wal().log(new PageSnapshot(new FullPageId(-1, -1), new byte[pageSize],
+                pageSize));
         }
 
         // WAL archive segment is allowed to be compressed when it's at least one checkpoint away from current WAL head.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java
index 300b752..3a99236 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalDeletionArchiveAbstractTest.java
@@ -55,7 +55,6 @@ public abstract class WalDeletionArchiveAbstractTest extends GridCommonAbstractT
         dbCfg.setWalMode(walMode());
         dbCfg.setWalSegmentSize(512 * 1024);
         dbCfg.setCheckpointFrequency(60 * 1000);//too high value for turn off frequency checkpoint.
-        dbCfg.setPageSize(4 * 1024);
         dbCfg.setDefaultDataRegionConfiguration(new DataRegionConfiguration()
             .setMaxSize(100 * 1024 * 1024)
             .setPersistenceEnabled(true));

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java
index 9a23502..14013b1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java
@@ -73,8 +73,6 @@ public abstract class IgniteDbAbstractTest extends GridCommonAbstractTest {
 
         if (isLargePage())
             dbCfg.setPageSize(16 * 1024);
-        else
-            dbCfg.setPageSize(4 * 1024);
 
         dbCfg.setWalMode(WALMode.LOG_ONLY);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
index 8ac72f7..02857ae 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.store.CacheStore;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheAffinitySharedManager;
+import org.apache.ignite.internal.processors.cache.CacheCompressionManager;
 import org.apache.ignite.internal.processors.cache.CacheOsConflictResolutionManager;
 import org.apache.ignite.internal.processors.cache.CacheType;
 import org.apache.ignite.internal.processors.cache.GridCacheAffinityManager;
@@ -90,6 +91,7 @@ public class GridCacheTestContext<K, V> extends GridCacheContext<K, V> {
             true,
             true,
             false,
+            new CacheCompressionManager(),
             new GridCacheEventManager(),
             new CacheOsStoreManager(null, new CacheConfiguration()),
             new GridCacheEvictionManager(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/direct-io/pom.xml
----------------------------------------------------------------------
diff --git a/modules/direct-io/pom.xml b/modules/direct-io/pom.xml
index e460e67..a5e2841 100644
--- a/modules/direct-io/pom.xml
+++ b/modules/direct-io/pom.xml
@@ -59,6 +59,21 @@
         </dependency>
 
         <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-compress</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-compress</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
             <groupId>org.springframework</groupId>
             <artifactId>spring-beans</artifactId>
             <version>${spring.version}</version>
@@ -75,7 +90,7 @@
         <dependency>
             <groupId>net.java.dev.jna</groupId>
             <artifactId>jna</artifactId>
-            <version>4.5.0</version>
+            <version>${jna.version}</version>
         </dependency>
 
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIO.java
----------------------------------------------------------------------
diff --git a/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIO.java b/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIO.java
index 98fd99b..a37dcdb 100644
--- a/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIO.java
+++ b/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIO.java
@@ -33,6 +33,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicReferenceArray;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.internal.processors.compress.FileSystemUtils;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.NotNull;
@@ -46,19 +47,22 @@ public class AlignedBuffersDirectFileIO extends AbstractFileIO {
     /** Negative value for file offset: read/write starting from current file position */
     private static final int FILE_POS_USE_CURRENT = -1;
 
-    /** File system & linux block size. Minimal amount of data can be written using DirectIO. */
-    private final int fsBlockSize;
+    /** Minimal amount of data can be written using DirectIO. */
+    private final int ioBlockSize;
 
-    /** Durable memory Page size. Can have greater value than {@link #fsBlockSize}. */
+    /** Durable memory Page size. Can have greater value than {@link #ioBlockSize}. */
     private final int pageSize;
 
+    /** File system block size. */
+    private final int fsBlockSize;
+
     /** File. */
     private final File file;
 
     /** Logger. */
     private final IgniteLogger log;
 
-    /** Thread local with buffers with capacity = one page {@link #pageSize} and aligned using {@link #fsBlockSize}. */
+    /** Thread local with buffers with capacity = one page {@link #pageSize} and aligned using {@link #ioBlockSize}. */
     private ThreadLocal<ByteBuffer> tlbOnePageAligned;
 
     /** Managed aligned buffers. Used to check if buffer is applicable for direct IO our data should be copied. */
@@ -79,18 +83,18 @@ public class AlignedBuffersDirectFileIO extends AbstractFileIO {
     /**
      * Creates Direct File IO.
      *
-     * @param fsBlockSize FS/OS block size.
+     * @param ioBlockSize FS/OS block size.
      * @param pageSize Durable memory Page size.
      * @param file File to open.
      * @param modes Open options (flags).
      * @param tlbOnePageAligned Thread local with buffers with capacity = one page {@code pageSize} and aligned using
-     * {@code fsBlockSize}.
+     * {@code ioBlockSize}.
      * @param managedAlignedBuffers Managed aligned buffers map, used to check if buffer is known.
      * @param log Logger.
      * @throws IOException if file open failed.
      */
     AlignedBuffersDirectFileIO(
-        int fsBlockSize,
+        int ioBlockSize,
         int pageSize,
         File file,
         OpenOption[] modes,
@@ -99,7 +103,7 @@ public class AlignedBuffersDirectFileIO extends AbstractFileIO {
         IgniteLogger log)
         throws IOException {
         this.log = log;
-        this.fsBlockSize = fsBlockSize;
+        this.ioBlockSize = ioBlockSize;
         this.pageSize = pageSize;
         this.file = file;
         this.tlbOnePageAligned = tlbOnePageAligned;
@@ -126,6 +130,7 @@ public class AlignedBuffersDirectFileIO extends AbstractFileIO {
                         "(probably incompatible file system selected, for example, tmpfs): " + msg);
 
                     this.fd = fd;
+                    fsBlockSize = FileSystemUtils.getFileSystemBlockSize(fd);
 
                     return;
                 }
@@ -135,6 +140,7 @@ public class AlignedBuffersDirectFileIO extends AbstractFileIO {
         }
 
         this.fd = fd;
+        fsBlockSize = FileSystemUtils.getFileSystemBlockSize(fd);
     }
 
     /**
@@ -174,6 +180,21 @@ public class AlignedBuffersDirectFileIO extends AbstractFileIO {
     }
 
     /** {@inheritDoc} */
+    @Override public int getFileSystemBlockSize() {
+        return fsBlockSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSparseSize() {
+        return FileSystemUtils.getSparseFileSize(fd);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int punchHole(long position, int len) {
+        return (int)FileSystemUtils.punchHole(fd, position, len, fsBlockSize);
+    }
+
+    /** {@inheritDoc} */
     @Override public long position() throws IOException {
         long position = IgniteNativeIoLib.lseek(fdCheckOpened(), 0, IgniteNativeIoLib.SEEK_CUR);
 
@@ -202,11 +223,22 @@ public class AlignedBuffersDirectFileIO extends AbstractFileIO {
     @Override public int read(ByteBuffer destBuf, long filePosition) throws IOException {
         int size = checkSizeIsPadded(destBuf.remaining());
 
-        if (isKnownAligned(destBuf))
-            return readIntoAlignedBuffer(destBuf, filePosition);
+        return isKnownAligned(destBuf) ?
+            readIntoAlignedBuffer(destBuf, filePosition) :
+            readIntoUnalignedBuffer(destBuf, filePosition, size);
+    }
 
+    /**
+     * @param destBuf Destination aligned byte buffer.
+     * @param filePosition File position.
+     * @param size Buffer size to write, should be divisible by {@link #ioBlockSize}.
+     * @return Number of read bytes, possibly zero, or <tt>-1</tt> if the
+     *         given position is greater than or equal to the file's current size.
+     * @throws IOException If failed.
+     */
+    private int readIntoUnalignedBuffer(ByteBuffer destBuf, long filePosition, int size) throws IOException {
         boolean useTlb = size == pageSize;
-        ByteBuffer alignedBuf = useTlb ? tlbOnePageAligned.get() : AlignedBuffers.allocate(fsBlockSize, size);
+        ByteBuffer alignedBuf = useTlb ? tlbOnePageAligned.get() : AlignedBuffers.allocate(ioBlockSize, size);
 
         try {
             assert alignedBuf.position() == 0: "Temporary aligned buffer is in incorrect state: position is set incorrectly";
@@ -241,29 +273,46 @@ public class AlignedBuffersDirectFileIO extends AbstractFileIO {
 
     /** {@inheritDoc} */
     @Override public int write(ByteBuffer srcBuf, long filePosition) throws IOException {
-        int size = checkSizeIsPadded(srcBuf.remaining());
+        return isKnownAligned(srcBuf) ?
+            writeFromAlignedBuffer(srcBuf, filePosition) :
+            writeFromUnalignedBuffer(srcBuf, filePosition);
+    }
 
-        if (isKnownAligned(srcBuf))
-            return writeFromAlignedBuffer(srcBuf, filePosition);
+    /**
+     * @param srcBuf buffer to check if it is known buffer.
+     * @param filePosition File position.
+     * @return Number of written bytes.
+     * @throws IOException If failed.
+     */
+    private int writeFromUnalignedBuffer(ByteBuffer srcBuf, long filePosition) throws IOException {
+        int size = srcBuf.remaining();
 
-        boolean useTlb = size == pageSize;
-        ByteBuffer alignedBuf = useTlb ? tlbOnePageAligned.get() : AlignedBuffers.allocate(fsBlockSize, size);
+        boolean useTlb = size <= pageSize;
+        ByteBuffer alignedBuf = useTlb ? tlbOnePageAligned.get() : AlignedBuffers.allocate(ioBlockSize, size);
 
         try {
             assert alignedBuf.position() == 0 : "Temporary aligned buffer is in incorrect state: position is set incorrectly";
-            assert alignedBuf.limit() == size : "Temporary aligned buffer is in incorrect state: limit is set incorrectly";
+            assert alignedBuf.limit() >= size : "Temporary aligned buffer is in incorrect state: limit is set incorrectly";
 
             int initPos = srcBuf.position();
 
             alignedBuf.put(srcBuf);
             alignedBuf.flip();
 
-            srcBuf.position(initPos); // will update later from write results
+            int len = alignedBuf.remaining();
+
+            // Compressed buffer of wrong size can be passed here.
+            if (len % ioBlockSize != 0)
+                alignBufferLimit(alignedBuf);
 
             int written = writeFromAlignedBuffer(alignedBuf, filePosition);
 
-            if (written > 0)
-                srcBuf.position(initPos + written);
+            // Actual written length can be greater than the original buffer,
+            // since we artificially expanded it to have correctly aligned size.
+            if (written > len)
+                written = len;
+
+            srcBuf.position(initPos + written);
 
             return written;
         }
@@ -276,6 +325,17 @@ public class AlignedBuffersDirectFileIO extends AbstractFileIO {
     }
 
     /**
+     * @param buf Byte buffer to align.
+     */
+    private void alignBufferLimit(ByteBuffer buf) {
+        int len = buf.remaining();
+
+        int alignedLen = (len / ioBlockSize + 1) * ioBlockSize;
+
+        buf.limit(buf.limit() + alignedLen - len);
+    }
+
+    /**
      * Checks if we can run fast path: we got well known buffer is already aligned.
      *
      * @param srcBuf buffer to check if it is known buffer.
@@ -290,16 +350,16 @@ public class AlignedBuffersDirectFileIO extends AbstractFileIO {
     /**
      * Check if size is appropriate for aligned/direct IO.
      *
-     * @param size buffer size to write, should be divisible by {@link #fsBlockSize}.
+     * @param size buffer size to write, should be divisible by {@link #ioBlockSize}.
      * @return size from parameter.
      * @throws IOException if provided size can't be written using direct IO.
      */
     private int checkSizeIsPadded(int size) throws IOException {
-        if (size % fsBlockSize != 0) {
+        if (size % ioBlockSize != 0) {
             throw new IOException(
-                String.format("Unable to apply DirectIO for read/write buffer [%d] bytes on file system " +
+                String.format("Unable to apply DirectIO for read/write buffer [%d] bytes on " +
                     "block size [%d]. Consider setting %s.setPageSize(%d) or disable Direct IO.",
-                    size, fsBlockSize, DataStorageConfiguration.class.getSimpleName(), fsBlockSize));
+                    size, ioBlockSize, DataStorageConfiguration.class.getSimpleName(), ioBlockSize));
         }
 
         return size;
@@ -446,9 +506,9 @@ public class AlignedBuffersDirectFileIO extends AbstractFileIO {
         if (pos > buf.capacity())
             throw new BufferOverflowException();
 
-        if ((alignedPointer + pos) % fsBlockSize != 0) {
+        if ((alignedPointer + pos) % ioBlockSize != 0) {
             U.warn(log, String.format("IO Buffer Pointer [%d] and/or offset [%d] seems to be not aligned " +
-                "for FS block size [%d]. Direct IO may fail.", alignedPointer, buf.position(), fsBlockSize));
+                "for IO block size [%d]. Direct IO may fail.", alignedPointer, buf.position(), ioBlockSize));
         }
 
         return new Pointer(alignedPointer + pos);

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIOFactory.java
----------------------------------------------------------------------
diff --git a/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIOFactory.java b/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIOFactory.java
index 85a3a02..5d919fe 100644
--- a/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIOFactory.java
+++ b/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIOFactory.java
@@ -49,7 +49,7 @@ public class AlignedBuffersDirectFileIOFactory implements FileIOFactory {
     private final FileIOFactory backupFactory;
 
     /** File system/os block size, negative value if library init was failed. */
-    private final int fsBlockSize;
+    private final int ioBlockSize;
 
     /** Use backup factory, {@code true} if direct IO setup failed. */
     private boolean useBackupFactory;
@@ -81,22 +81,22 @@ public class AlignedBuffersDirectFileIOFactory implements FileIOFactory {
         this.backupFactory = backupFactory;
 
         useBackupFactory = true;
-        fsBlockSize = IgniteNativeIoLib.getFsBlockSize(storePath.getAbsolutePath(), log);
+        ioBlockSize = IgniteNativeIoLib.getDirectIOBlockSize(storePath.getAbsolutePath(), log);
 
         if(!IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_DIRECT_IO_ENABLED, true)) {
             if (log.isInfoEnabled())
-                log.info("Direct IO is explicitly disabled by system property");
+                log.info("Direct IO is explicitly disabled by system property.");
 
             return;
         }
 
-        if (fsBlockSize > 0) {
-            int blkSize = fsBlockSize;
+        if (ioBlockSize > 0) {
+            int blkSize = ioBlockSize;
 
             if (pageSize % blkSize != 0) {
                 U.warn(log, String.format("Unable to setup Direct IO for Ignite [pageSize=%d bytes;" +
                         " file system block size=%d]. For speeding up Ignite consider setting %s.setPageSize(%d)." +
-                        " Direct IO is disabled",
+                        " Direct IO is disabled.",
                     pageSize, blkSize, DataStorageConfiguration.class.getSimpleName(), blkSize));
             }
             else {
@@ -133,7 +133,7 @@ public class AlignedBuffersDirectFileIOFactory implements FileIOFactory {
         assert !useBackupFactory : "Direct IO is disabled, aligned managed buffer creation is disabled now";
         assert managedAlignedBuffers != null : "Direct buffers not available";
 
-        ByteBuffer allocate = AlignedBuffers.allocate(fsBlockSize, size).order(ByteOrder.nativeOrder());
+        ByteBuffer allocate = AlignedBuffers.allocate(ioBlockSize, size).order(ByteOrder.nativeOrder());
 
         managedAlignedBuffers.put(GridUnsafe.bufferAddress(allocate), Thread.currentThread());
 
@@ -145,7 +145,7 @@ public class AlignedBuffersDirectFileIOFactory implements FileIOFactory {
         if (useBackupFactory)
             return backupFactory.create(file, modes);
 
-        return new AlignedBuffersDirectFileIO(fsBlockSize, pageSize, file, modes, tlbOnePageAligned, managedAlignedBuffers, log);
+        return new AlignedBuffersDirectFileIO(ioBlockSize, pageSize, file, modes, tlbOnePageAligned, managedAlignedBuffers, log);
 
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/IgniteNativeIoLib.java
----------------------------------------------------------------------
diff --git a/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/IgniteNativeIoLib.java b/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/IgniteNativeIoLib.java
index 65ef8d7..2ab4325 100644
--- a/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/IgniteNativeIoLib.java
+++ b/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/IgniteNativeIoLib.java
@@ -190,7 +190,7 @@ public class IgniteNativeIoLib {
      * <li>and <tt>-1</tt> if failed to determine block size.</li>
      * <li>and <tt>-1</tt> if JNA is not available or init failed.</li> </ul>
      */
-    public static int getFsBlockSize(final String storageDir, final IgniteLogger log) {
+    public static int getDirectIOBlockSize(final String storageDir, final IgniteLogger log) {
         if (ex != null) {
             U.warn(log, "Failed to initialize O_DIRECT support at current OS: " + ex.getMessage(), ex);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/direct-io/src/test/java/org/apache/ignite/internal/processors/cache/persistence/DiskPageCompressionIntegrationDirectIOTest.java
----------------------------------------------------------------------
diff --git a/modules/direct-io/src/test/java/org/apache/ignite/internal/processors/cache/persistence/DiskPageCompressionIntegrationDirectIOTest.java b/modules/direct-io/src/test/java/org/apache/ignite/internal/processors/cache/persistence/DiskPageCompressionIntegrationDirectIOTest.java
new file mode 100644
index 0000000..89b963a
--- /dev/null
+++ b/modules/direct-io/src/test/java/org/apache/ignite/internal/processors/cache/persistence/DiskPageCompressionIntegrationDirectIOTest.java
@@ -0,0 +1,31 @@
+/*
+ * 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.internal.processors.cache.persistence;
+
+import org.apache.ignite.internal.processors.cache.persistence.file.AlignedBuffersDirectFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.compress.DiskPageCompressionIntegrationTest;
+
+/**
+ */
+public class DiskPageCompressionIntegrationDirectIOTest extends DiskPageCompressionIntegrationTest {
+    /** {@inheritDoc} */
+    @Override protected void checkFileIOFactory(FileIOFactory f) {
+        assertTrue(f instanceof AlignedBuffersDirectFileIOFactory);
+    }
+}


[42/50] [abbrv] ignite git commit: IGNITE-10429: ML: TensorFlowLocalInferenceExample fails on Windows

Posted by ag...@apache.org.
IGNITE-10429: ML: TensorFlowLocalInferenceExample fails on Windows

This closes #5514


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

Branch: refs/heads/ignite-9720
Commit: 3885f3f16c159c841a9219942156d12968f5f9d6
Parents: 566adaa
Author: Anton Dmitriev <dm...@gmail.com>
Authored: Wed Nov 28 00:02:09 2018 +0300
Committer: Yury Babak <yb...@gridgain.com>
Committed: Wed Nov 28 00:02:09 2018 +0300

----------------------------------------------------------------------
 .../inference/util/DirectorySerializerTest.java   | 18 ++++++++++--------
 parent/pom.xml                                    |  1 +
 2 files changed, 11 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3885f3f1/modules/ml/src/test/java/org/apache/ignite/ml/inference/util/DirectorySerializerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/inference/util/DirectorySerializerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/inference/util/DirectorySerializerTest.java
index d2d6b16..87a515c 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/inference/util/DirectorySerializerTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/inference/util/DirectorySerializerTest.java
@@ -63,10 +63,11 @@ public class DirectorySerializerTest {
             assertEquals(1, files.length);
             assertEquals("test.txt", files[0].getName());
 
-            Scanner scanner = new Scanner(files[0]);
-            assertTrue(scanner.hasNextLine());
-            assertEquals("Hello, world!", scanner.nextLine());
-            assertFalse(scanner.hasNextLine());
+            try (Scanner scanner = new Scanner(files[0])) {
+                assertTrue(scanner.hasNextLine());
+                assertEquals("Hello, world!", scanner.nextLine());
+                assertFalse(scanner.hasNextLine());
+            }
         }
         finally {
             DirectorySerializer.deleteDirectory(src);
@@ -111,10 +112,11 @@ public class DirectorySerializerTest {
             assertEquals(1, files.length);
             assertEquals("test.txt", files[0].getName());
 
-            Scanner scanner = new Scanner(files[0]);
-            assertTrue(scanner.hasNextLine());
-            assertEquals("Hello, world!", scanner.nextLine());
-            assertFalse(scanner.hasNextLine());
+            try (Scanner scanner = new Scanner(files[0])) {
+                assertTrue(scanner.hasNextLine());
+                assertEquals("Hello, world!", scanner.nextLine());
+                assertFalse(scanner.hasNextLine());
+            }
         }
         finally {
             DirectorySerializer.deleteDirectory(src);

http://git-wip-us.apache.org/repos/asf/ignite/blob/3885f3f1/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 23e0a0d..2032f1d 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -878,6 +878,7 @@
                                         <exclude>src/main/java/org/apache/ignite/examples/streaming/wordcount/*.txt</exclude><!--books examples-->
                                         <exclude>examples/src/main/java/org/apache/ignite/examples/streaming/wordcount/*.txt</exclude><!--books examples-->
                                         <exclude>**/resources/datasets/**/*</exclude><!--Datasets in ml module-->
+                                        <exclude>**/*.pb</exclude><!--Pre-trained TF model-->
                                         <exclude>src/main/java/org/jetbrains/annotations/*.java</exclude><!--copyright-->
                                         <exclude>dev-tools/IGNITE-*.patch</exclude>
                                         <exclude>dev-tools/.gradle/**/*</exclude>


[25/50] [abbrv] ignite git commit: IGNITE-9442 Collocated IgniteSet#close is not working on non-affinity node.

Posted by ag...@apache.org.
IGNITE-9442 Collocated IgniteSet#close is not working on non-affinity node.

Signed-off-by: Anton Vinogradov <av...@apache.org>


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

Branch: refs/heads/ignite-9720
Commit: 507aeb21bf6792bb9efb6518f83049ebc7eab53f
Parents: 6fd6c32
Author: pereslegin-pa <xx...@gmail.com>
Authored: Mon Nov 26 18:53:10 2018 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Mon Nov 26 18:53:10 2018 +0300

----------------------------------------------------------------------
 .../CacheDataStructuresManager.java             | 15 +++---
 .../datastructures/GridCacheSetImpl.java        |  4 +-
 .../GridCacheSetAbstractSelfTest.java           | 31 ++++++++++--
 ...idCachePartitionedSetWithClientSelfTest.java | 52 ++++++++++++++++++++
 ...chePartitionedSetWithNodeFilterSelfTest.java | 37 ++++++++++++++
 ...ridCacheReplicatedSetWithClientSelfTest.java | 52 ++++++++++++++++++++
 ...acheReplicatedSetWithNodeFilterSelfTest.java | 37 ++++++++++++++
 .../IgniteCacheDataStructuresSelfTestSuite.java |  8 +++
 8 files changed, 224 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/507aeb21/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
index 932f000..9c3c4a6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
@@ -63,6 +63,7 @@ import org.apache.ignite.internal.processors.datastructures.GridTransactionalCac
 import org.apache.ignite.internal.processors.datastructures.SetItemKey;
 import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteFuture;
@@ -472,7 +473,7 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
             while (true) {
                 AffinityTopologyVersion topVer = cctx.topologyVersionFuture().get();
 
-                Collection<ClusterNode> nodes = CU.affinityNodes(cctx, topVer);
+                Collection<ClusterNode> nodes = F.view(cctx.discovery().nodes(topVer), node -> !node.isDaemon());
 
                 try {
                     cctx.closures().callAsyncNoFailover(BROADCAST,
@@ -502,10 +503,12 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
                         throw e;
                 }
 
+                Collection<ClusterNode> affNodes = CU.affinityNodes(cctx, topVer);
+
                 try {
                     cctx.closures().callAsyncNoFailover(BROADCAST,
                         new RemoveSetDataCallable(cctx.name(), id, topVer),
-                        nodes,
+                        affNodes,
                         true,
                         0, false).get();
                 }
@@ -516,7 +519,7 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
 
                         continue;
                     }
-                    else if (!pingNodes(nodes)) {
+                    else if (!pingNodes(affNodes)) {
                         if (log.isDebugEnabled())
                             log.debug("RemoveSetData job failed and set data node left, will retry: " + e);
 
@@ -650,9 +653,9 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
 
             GridCacheAdapter cache = ((IgniteKernal)ignite).context().cache().internalCache(cacheName);
 
-            assert cache != null : cacheName;
-
-            cache.context().dataStructures().blockSet(setId);
+            // On non-affinity node cache starts on demand, so it may not be running.
+            if (cache != null)
+                cache.context().dataStructures().blockSet(setId);
 
             return null;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/507aeb21/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
index 8aae9d0..97d34f0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
@@ -514,7 +514,9 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
      * @throws IgniteCheckedException If all cache nodes left grid.
      */
     private Collection<ClusterNode> dataNodes(AffinityTopologyVersion topVer) throws IgniteCheckedException {
-        if (ctx.isLocal() || ctx.isReplicated())
+        assert ctx.isPartitioned() || collocated : "Non-collocated mode is supported only for PARTITIONED caches.";
+
+        if (ctx.isLocal() || (ctx.isReplicated() && ctx.affinityNode()))
             return Collections.singleton(ctx.localNode());
 
         Collection<ClusterNode> nodes;

http://git-wip-us.apache.org/repos/asf/ignite/blob/507aeb21/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java
index fef55b8..4e888aa 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java
@@ -32,6 +32,7 @@ import junit.framework.AssertionFailedError;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteCluster;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteSet;
 import org.apache.ignite.cache.CacheMode;
@@ -181,13 +182,22 @@ public abstract class GridCacheSetAbstractSelfTest extends IgniteCollectionAbstr
      * @param collocated Collocation flag.
      * @throws Exception If failed.
      */
-    private void testCreateRemove(boolean collocated) throws Exception {
+    protected void testCreateRemove(boolean collocated) throws Exception {
+        testCreateRemove(collocated, 0);
+    }
+
+    /**
+     * @param collocated Collocation flag.
+     * @param nodeIdx Index of the node from which to create set.
+     * @throws Exception If failed.
+     */
+    protected void testCreateRemove(boolean collocated, int nodeIdx) throws Exception {
         for (int i = 0; i < gridCount(); i++)
             assertNull(grid(i).set(SET_NAME, null));
 
         CollectionConfiguration colCfg0 = config(collocated);
 
-        IgniteSet<Integer> set0 = grid(0).set(SET_NAME, colCfg0);
+        IgniteSet<Integer> set0 = grid(nodeIdx).set(SET_NAME, colCfg0);
 
         assertNotNull(set0);
 
@@ -411,10 +421,19 @@ public abstract class GridCacheSetAbstractSelfTest extends IgniteCollectionAbstr
      * @param collocated Collocation flag.
      * @throws Exception If failed.
      */
-    private void testIterator(boolean collocated) throws Exception {
+    protected void testIterator(boolean collocated) throws Exception {
+        testIterator(collocated, 0);
+    }
+
+    /**
+     * @param collocated Collocation flag.
+     * @param nodeIdx Index of the node from which to create set.
+     * @throws Exception If failed.
+     */
+    protected void testIterator(boolean collocated, int nodeIdx) throws Exception {
         CollectionConfiguration colCfg = config(collocated);
 
-        final IgniteSet<Integer> set0 = grid(0).set(SET_NAME, colCfg);
+        final IgniteSet<Integer> set0 = grid(nodeIdx).set(SET_NAME, colCfg);
 
         for (int i = 0; i < gridCount(); i++) {
             IgniteSet<Integer> set = grid(i).set(SET_NAME, null);
@@ -839,7 +858,9 @@ public abstract class GridCacheSetAbstractSelfTest extends IgniteCollectionAbstr
         for (int i = 0; i < 10; i++)
             set.add(i);
 
-        Collection<Integer> c = grid(0).compute().broadcast(new IgniteCallable<Integer>() {
+        IgniteCluster cluster = grid(0).cluster();
+
+        Collection<Integer> c = grid(0).compute(cluster).broadcast(new IgniteCallable<Integer>() {
             @Override public Integer call() throws Exception {
                 assertEquals(SET_NAME, set.name());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/507aeb21/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedSetWithClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedSetWithClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedSetWithClientSelfTest.java
new file mode 100644
index 0000000..97be50d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedSetWithClientSelfTest.java
@@ -0,0 +1,52 @@
+/*
+ * 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.internal.processors.cache.datastructures.partitioned;
+
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ * Tests IgniteSet with client node on {@code PARTITIONED} cache.
+ */
+public class GridCachePartitionedSetWithClientSelfTest extends GridCachePartitionedSetSelfTest {
+    /** */
+    private static final int CLIENT_NODE_IDX = 4;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setClientMode(getTestIgniteInstanceIndex(igniteInstanceName) == CLIENT_NODE_IDX);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void testCreateRemove(boolean collocated) throws Exception {
+        testCreateRemove(collocated, CLIENT_NODE_IDX);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void testIterator(boolean collocated) throws Exception {
+        testIterator(collocated, CLIENT_NODE_IDX);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 5;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/507aeb21/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedSetWithNodeFilterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedSetWithNodeFilterSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedSetWithNodeFilterSelfTest.java
new file mode 100644
index 0000000..116f383
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedSetWithNodeFilterSelfTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.internal.processors.cache.datastructures.partitioned;
+
+import org.apache.ignite.configuration.CollectionConfiguration;
+import org.apache.ignite.util.AttributeNodeFilter;
+
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGNITE_INSTANCE_NAME;
+
+/**
+ * Tests IgniteSet with node filter on {@code PARTITIONED} cache.
+ */
+public class GridCachePartitionedSetWithNodeFilterSelfTest extends GridCachePartitionedSetSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CollectionConfiguration collectionConfiguration() {
+        CollectionConfiguration cfg = super.collectionConfiguration();
+
+        cfg.setNodeFilter(new AttributeNodeFilter(ATTR_IGNITE_INSTANCE_NAME, getTestIgniteInstanceName(0)));
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/507aeb21/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedSetWithClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedSetWithClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedSetWithClientSelfTest.java
new file mode 100644
index 0000000..0c975ac
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedSetWithClientSelfTest.java
@@ -0,0 +1,52 @@
+/*
+ * 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.internal.processors.cache.datastructures.replicated;
+
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ * Tests IgniteSet with client node on {@code REPLICATED} cache.
+ */
+public class GridCacheReplicatedSetWithClientSelfTest extends GridCacheReplicatedSetSelfTest {
+    /** */
+    private static final int CLIENT_NODE_IDX = 4;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setClientMode(getTestIgniteInstanceIndex(igniteInstanceName) == CLIENT_NODE_IDX);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void testCreateRemove(boolean collocated) throws Exception {
+        testCreateRemove(collocated, CLIENT_NODE_IDX);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void testIterator(boolean collocated) throws Exception {
+        testIterator(collocated, CLIENT_NODE_IDX);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 5;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/507aeb21/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedSetWithNodeFilterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedSetWithNodeFilterSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedSetWithNodeFilterSelfTest.java
new file mode 100644
index 0000000..1db790b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/replicated/GridCacheReplicatedSetWithNodeFilterSelfTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.internal.processors.cache.datastructures.replicated;
+
+import org.apache.ignite.configuration.CollectionConfiguration;
+import org.apache.ignite.util.AttributeNodeFilter;
+
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGNITE_INSTANCE_NAME;
+
+/**
+ * Tests IgniteSet with node filter on {@code REPLICATED} cache.
+ */
+public class GridCacheReplicatedSetWithNodeFilterSelfTest extends GridCacheReplicatedSetSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CollectionConfiguration collectionConfiguration() {
+        CollectionConfiguration cfg = super.collectionConfiguration();
+
+        cfg.setNodeFilter(new AttributeNodeFilter(ATTR_IGNITE_INSTANCE_NAME, getTestIgniteInstanceName(0)));
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/507aeb21/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
index db74c72..bca706e 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
@@ -62,6 +62,8 @@ import org.apache.ignite.internal.processors.cache.datastructures.partitioned.Gr
 import org.apache.ignite.internal.processors.cache.datastructures.partitioned.GridCachePartitionedSequenceMultiNodeSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.partitioned.GridCachePartitionedSetFailoverSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.partitioned.GridCachePartitionedSetSelfTest;
+import org.apache.ignite.internal.processors.cache.datastructures.partitioned.GridCachePartitionedSetWithClientSelfTest;
+import org.apache.ignite.internal.processors.cache.datastructures.partitioned.GridCachePartitionedSetWithNodeFilterSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.partitioned.IgnitePartitionedAtomicLongApiSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.partitioned.IgnitePartitionedCountDownLatchSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.partitioned.IgnitePartitionedQueueNoBackupsTest;
@@ -76,6 +78,8 @@ import org.apache.ignite.internal.processors.cache.datastructures.replicated.Gri
 import org.apache.ignite.internal.processors.cache.datastructures.replicated.GridCacheReplicatedSequenceApiSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.replicated.GridCacheReplicatedSequenceMultiNodeSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.replicated.GridCacheReplicatedSetSelfTest;
+import org.apache.ignite.internal.processors.cache.datastructures.replicated.GridCacheReplicatedSetWithClientSelfTest;
+import org.apache.ignite.internal.processors.cache.datastructures.replicated.GridCacheReplicatedSetWithNodeFilterSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.replicated.IgniteReplicatedAtomicLongApiSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.replicated.IgniteReplicatedCountDownLatchSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.replicated.IgniteReplicatedLockSelfTest;
@@ -112,6 +116,8 @@ public class IgniteCacheDataStructuresSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(GridCacheReplicatedQueueMultiNodeSelfTest.class));
         suite.addTest(new TestSuite(GridCacheReplicatedQueueRotativeMultiNodeTest.class));
         suite.addTest(new TestSuite(GridCacheReplicatedSetSelfTest.class));
+        suite.addTest(new TestSuite(GridCacheReplicatedSetWithClientSelfTest.class));
+        suite.addTest(new TestSuite(GridCacheReplicatedSetWithNodeFilterSelfTest.class));
         suite.addTest(new TestSuite(GridCacheReplicatedDataStructuresFailoverSelfTest.class));
         suite.addTest(new TestSuite(IgniteReplicatedCountDownLatchSelfTest.class));
         suite.addTest(new TestSuite(IgniteReplicatedSemaphoreSelfTest.class));
@@ -129,6 +135,8 @@ public class IgniteCacheDataStructuresSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(GridCachePartitionedQueueCreateMultiNodeSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedAtomicQueueCreateMultiNodeSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedSetSelfTest.class));
+        suite.addTest(new TestSuite(GridCachePartitionedSetWithClientSelfTest.class));
+        suite.addTest(new TestSuite(GridCachePartitionedSetWithNodeFilterSelfTest.class));
         suite.addTest(new TestSuite(IgnitePartitionedSetNoBackupsSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedAtomicSetSelfTest.class));
         suite.addTest(new TestSuite(IgnitePartitionedCountDownLatchSelfTest.class));


[40/50] [abbrv] ignite git commit: IGNITE-10158 Some tests in IgniteCacheAbstractQuerySelfTest are incorrectly muted - Fixes #5344.

Posted by ag...@apache.org.
IGNITE-10158 Some tests in IgniteCacheAbstractQuerySelfTest are incorrectly muted - Fixes #5344.

Signed-off-by: Pavel Kovalenko <jo...@gmail.com>


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

Branch: refs/heads/ignite-9720
Commit: be97bcdcfa6f0ae0bc37d156806bbac4c09259dc
Parents: 07f1dbf
Author: Oleg Ignatenko <oi...@gridgain.com>
Authored: Tue Nov 27 20:02:47 2018 +0300
Committer: Pavel Kovalenko <jo...@gmail.com>
Committed: Tue Nov 27 20:02:47 2018 +0300

----------------------------------------------------------------------
 .../cache/IgniteCacheAbstractQuerySelfTest.java | 54 +++-----------------
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |  6 +--
 2 files changed, 8 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/be97bcdc/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
index 1c1b879..d5b8ccc 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
@@ -286,21 +286,12 @@ public abstract class IgniteCacheAbstractQuerySelfTest extends GridCommonAbstrac
      *
      * @throws Exception In case of error.
      */
-    public void _testDifferentKeyTypes() throws Exception {
-        fail("http://atlassian.gridgain.com/jira/browse/GG-11216");
-
+    public void testDifferentKeyTypes() throws Exception {
         final IgniteCache<Object, Object> cache = jcache(Object.class, Object.class);
 
         cache.put(1, "value");
 
-        try {
-            cache.put("key", "value");
-
-            fail();
-        }
-        catch (CacheException ignored) {
-            // No-op.
-        }
+        cache.put("key", "value");
     }
 
     /**
@@ -521,7 +512,7 @@ public abstract class IgniteCacheAbstractQuerySelfTest extends GridCommonAbstrac
 
         QueryCursor<Cache.Entry<Key, GridCacheQueryTestValue>> qry = cache
             .query(new SqlQuery<Key, GridCacheQueryTestValue>(GridCacheQueryTestValue.class,
-                    "fieldName='field1' and field2=1 and field3=1 and id=100500 and embeddedField2=11 and x=3"));
+                "fieldName='field1' and field2=1 and field3=1 and id=100500 and embeddedField2=11 and x=3"));
 
         Cache.Entry<Key, GridCacheQueryTestValue> entry = F.first(qry.getAll());
 
@@ -2232,39 +2223,6 @@ public abstract class IgniteCacheAbstractQuerySelfTest extends GridCommonAbstrac
     }
 
     /**
-     * Empty test object.
-     */
-    private static class EmptyObject {
-        /** */
-        private int val;
-
-        /**
-         * @param val Value.
-         */
-        private EmptyObject(int val) {
-            this.val = val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (!(o instanceof EmptyObject))
-                return false;
-
-            EmptyObject that = (EmptyObject)o;
-
-            return val == that.val;
-        }
-    }
-
-    /**
      *
      */
     private static class BadHashKeyObject implements Serializable, Comparable<BadHashKeyObject> {
@@ -2404,9 +2362,9 @@ public abstract class IgniteCacheAbstractQuerySelfTest extends GridCommonAbstrac
          */
         @Override public String toString() {
             return "EnumObject{" +
-                    "id=" + id +
-                    ", type=" + type +
-                    '}';
+                "id=" + id +
+                ", type=" + type +
+                '}';
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/be97bcdc/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
index 8f28501..e9ebc18 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
@@ -360,9 +360,7 @@ public class IgniteCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTes
      * @throws Exception If failed.
      */
     @SuppressWarnings({"TooBroadScope"})
-    public void _testMultiThreadedSwapUnswapLongString() throws Exception {
-        fail("http://atlassian.gridgain.com/jira/browse/GG-11216");
-
+    public void testMultiThreadedSwapUnswapLongString() throws Exception {
         int threadCnt = 50;
         final int keyCnt = 2000;
         final int valCnt = 10000;
@@ -758,4 +756,4 @@ public class IgniteCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTes
             return val;
         }
     }
-}
\ No newline at end of file
+}


[29/50] [abbrv] ignite git commit: IGNITE-10002: MVCC: Create "Cache 2" test suite for MVCC mode. This closes #5198.

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
index ce8749c..dd6e36e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
@@ -86,6 +86,7 @@ import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -140,6 +141,13 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
     }
 
     /** {@inheritDoc} */
+    @Override public void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         super.afterTest();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxIteratorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxIteratorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxIteratorSelfTest.java
index c211590..e5bb583 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxIteratorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxIteratorSelfTest.java
@@ -27,8 +27,9 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.configuration.TransactionConfiguration;
-import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.testframework.MvccFeatureChecker;
+import org.apache.ignite.testframework.MvccFeatureChecker.Feature;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -105,14 +106,14 @@ public class IgniteCacheTxIteratorSelfTest extends GridCommonAbstractTest {
         try {
             for (CacheMode mode : CacheMode.values()) {
                 for (CacheAtomicityMode atomMode : CacheAtomicityMode.values()) {
-                    if (mode == CacheMode.PARTITIONED && atomMode != CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT) {
+                    if (mode == CacheMode.PARTITIONED) {
                         // Near cache makes sense only for partitioned cache.
                         checkTxCache(CacheMode.PARTITIONED, atomMode, true, false);
                     }
 
-                    checkTxCache(CacheMode.PARTITIONED, atomMode, false, true);
+                    checkTxCache(mode, atomMode, false, true);
 
-                    checkTxCache(CacheMode.PARTITIONED, atomMode, false, false);
+                    checkTxCache(mode, atomMode, false, false);
                 }
             }
         }
@@ -130,6 +131,13 @@ public class IgniteCacheTxIteratorSelfTest extends GridCommonAbstractTest {
         boolean nearEnabled,
         boolean useEvicPlc
     ) throws Exception {
+        if (atomMode == CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT) {
+            if (!MvccFeatureChecker.isSupported(mode) ||
+                (nearEnabled && !MvccFeatureChecker.isSupported(Feature.NEAR_CACHE)) ||
+                (useEvicPlc && !MvccFeatureChecker.isSupported(Feature.EVICTION)))
+                return; // Nothing to do. Mode is not supported.
+        }
+
         final Ignite ignite = grid(0);
 
         final CacheConfiguration<String, TestClass> ccfg = cacheConfiguration(
@@ -154,14 +162,11 @@ public class IgniteCacheTxIteratorSelfTest extends GridCommonAbstractTest {
 
                 for (TransactionIsolation iso : TransactionIsolation.values()) {
                     for (TransactionConcurrency con : TransactionConcurrency.values()) {
-                        try (Transaction transaction = ignite.transactions().txStart(con, iso)) {
-                            //TODO: IGNITE-7187: Fix when ticket will be implemented. (Near cache)
-                            //TODO: IGNITE-7956: Fix when ticket will be implemented. (Eviction)
-                            if (((IgniteCacheProxy)cache).context().mvccEnabled() &&
-                                ((iso != TransactionIsolation.REPEATABLE_READ && con != TransactionConcurrency.PESSIMISTIC)
-                                    || nearEnabled || useEvicPlc))
-                                return; // Nothing to do. Mode is not supported.
+                        if (atomMode == CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT &&
+                            !MvccFeatureChecker.isSupported(con, iso))
+                            continue; // Mode not supported.
 
+                        try (Transaction transaction = ignite.transactions().txStart(con, iso)) {
                             assertEquals(val, cache.get(key));
 
                             transaction.commit();

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteMvccTxTimeoutAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteMvccTxTimeoutAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteMvccTxTimeoutAbstractTest.java
new file mode 100644
index 0000000..15d23f4
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteMvccTxTimeoutAbstractTest.java
@@ -0,0 +1,146 @@
+/*
+ * 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.internal.processors.cache.distributed;
+
+import java.util.Random;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.TransactionConfiguration;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+import org.apache.ignite.transactions.TransactionTimeoutException;
+
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Simple cache test.
+ */
+public class IgniteMvccTxTimeoutAbstractTest extends GridCommonAbstractTest {
+    /** Random number generator. */
+    private static final Random RAND = new Random();
+
+    /** Grid count. */
+    private static final int GRID_COUNT = 2;
+
+    /** Transaction timeout. */
+    private static final long TIMEOUT = 50;
+
+    /** */
+    private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Override protected void beforeTestsStarted() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-7388");
+
+        startGridsMultiThreaded(GRID_COUNT, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
+
+        TransactionConfiguration txCfg = c.getTransactionConfiguration();
+
+        txCfg.setDefaultTxTimeout(TIMEOUT);
+
+        TcpDiscoverySpi spi = new TcpDiscoverySpi();
+
+        spi.setIpFinder(ipFinder);
+
+        c.setDiscoverySpi(spi);
+
+        return c;
+    }
+
+    /**
+     * @param i Grid index.
+     * @return Cache.
+     */
+    @Override protected <K, V> IgniteCache<K, V> jcache(int i) {
+        return grid(i).cache(DEFAULT_CACHE_NAME);
+    }
+
+    /**
+     * @throws IgniteCheckedException If test failed.
+     */
+    public void testPessimisticRepeatableRead() throws Exception {
+        checkTransactionTimeout(PESSIMISTIC, REPEATABLE_READ);
+    }
+
+    /**
+     * @param concurrency Concurrency.
+     * @param isolation Isolation.
+     * @throws IgniteCheckedException If test failed.
+     */
+    private void checkTransactionTimeout(TransactionConcurrency concurrency,
+        TransactionIsolation isolation) throws Exception {
+        int idx = RAND.nextInt(GRID_COUNT);
+
+        IgniteCache<Integer, String> cache = jcache(idx);
+
+        Transaction tx = ignite(idx).transactions().txStart(concurrency, isolation, TIMEOUT, 0);
+
+        try {
+            info("Storing value in cache [key=1, val=1]");
+
+            cache.put(1, "1");
+
+            long sleep = TIMEOUT * 2;
+
+            info("Going to sleep for (ms): " + sleep);
+
+            Thread.sleep(sleep);
+
+            info("Storing value in cache [key=1, val=2]");
+
+            cache.put(1, "2");
+
+            info("Committing transaction: " + tx);
+
+            tx.commit();
+
+            assert false : "Timeout never happened for transaction: " + tx;
+        }
+        catch (Exception e) {
+            if (!(X.hasCause(e, TransactionTimeoutException.class)))
+                throw e;
+
+            info("Received expected timeout exception [msg=" + e.getMessage() + ", tx=" + tx + ']');
+        }
+        finally {
+            tx.close();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxTimeoutAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxTimeoutAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxTimeoutAbstractTest.java
index a0ec70a..4cd64e3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxTimeoutAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxTimeoutAbstractTest.java
@@ -17,13 +17,15 @@
 
 package org.apache.ignite.internal.processors.cache.distributed;
 
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Random;
-import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.TransactionConfiguration;
 import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -46,25 +48,41 @@ public class IgniteTxTimeoutAbstractTest extends GridCommonAbstractTest {
     /** Grid count. */
     private static final int GRID_COUNT = 2;
 
-    /** Grid instances. */
-    private static final List<Ignite> IGNITEs = new ArrayList<>();
-
     /** Transaction timeout. */
     private static final long TIMEOUT = 50;
 
+    /** */
+    private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
     /**
      * @throws Exception If failed.
      */
     @Override protected void beforeTestsStarted() throws Exception {
-        for (int i = 0; i < GRID_COUNT; i++)
-            IGNITEs.add(startGrid(i));
+        startGridsMultiThreaded(GRID_COUNT);
     }
 
     /**
      * @throws Exception If failed.
      */
     @Override protected void afterTestsStopped() throws Exception {
-        IGNITEs.clear();
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
+
+        TransactionConfiguration txCfg = c.getTransactionConfiguration();
+
+        txCfg.setDefaultTxTimeout(TIMEOUT);
+
+        TcpDiscoverySpi spi = new TcpDiscoverySpi();
+
+        spi.setIpFinder(ipFinder);
+
+        c.setDiscoverySpi(spi);
+
+        return c;
     }
 
     /**
@@ -72,7 +90,7 @@ public class IgniteTxTimeoutAbstractTest extends GridCommonAbstractTest {
      * @return Cache.
      */
     @Override protected <K, V> IgniteCache<K, V> jcache(int i) {
-        return IGNITEs.get(i).cache(DEFAULT_CACHE_NAME);
+        return grid(i).cache(DEFAULT_CACHE_NAME);
     }
 
     /**
@@ -124,7 +142,6 @@ public class IgniteTxTimeoutAbstractTest extends GridCommonAbstractTest {
      */
     private void checkTransactionTimeout(TransactionConcurrency concurrency,
         TransactionIsolation isolation) throws Exception {
-
         int idx = RAND.nextInt(GRID_COUNT);
 
         IgniteCache<Integer, String> cache = jcache(idx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedMvccTxSingleThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedMvccTxSingleThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedMvccTxSingleThreadedSelfTest.java
new file mode 100644
index 0000000..717d771
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedMvccTxSingleThreadedSelfTest.java
@@ -0,0 +1,85 @@
+/*
+ * 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.internal.processors.cache.distributed.dht;
+
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.cache.IgniteMvccTxSingleThreadedAbstractTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheRebalanceMode.NONE;
+
+/**
+ * Test Mvcc txs in single-threaded mode for colocated cache.
+ */
+public class GridCacheColocatedMvccTxSingleThreadedSelfTest extends IgniteMvccTxSingleThreadedAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg = defaultCacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setNearConfiguration(null);
+        ccfg.setAtomicityMode(TRANSACTIONAL_SNAPSHOT);
+
+        ccfg.setEvictionPolicy(null);
+
+        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_ASYNC);
+
+        ccfg.setRebalanceMode(NONE);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 4;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int keyCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int maxKeyValue() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int iterations() {
+        return 3000;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean isTestDebug() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean printMemoryStats() {
+        return true;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
index 5aa6510..853763e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
@@ -37,6 +37,7 @@ import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 
@@ -78,6 +79,10 @@ public class GridCacheDhtEntrySelfTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
+        super.beforeTestsStarted();
+
         startGridsMultiThreaded(GRID_CNT);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtMappingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtMappingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtMappingSelfTest.java
index bd0af34..5456e4c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtMappingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtMappingSelfTest.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCach
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -44,6 +45,13 @@ public class GridCacheDhtMappingSelfTest extends GridCommonAbstractTest {
     private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
index 66efc18..d8bc44a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
@@ -40,6 +40,7 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -73,6 +74,13 @@ public class GridCacheDhtPreloadDisabledSelfTest extends GridCommonAbstractTest
     /** IP finder. */
     private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
+        super.beforeTestsStarted();
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
index 6c32a67..5f7be7a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
@@ -33,6 +33,7 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
@@ -137,6 +138,8 @@ public class GridCacheDhtPreloadMultiThreadedSelfTest extends GridCommonAbstract
      * @throws Exception If failed.
      */
     public void testConcurrentNodesStartStop() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.LOCAL_CACHE);
+
         try {
             multithreadedAsync(
                 new Callable<Object>() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadPutGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadPutGetSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadPutGetSelfTest.java
index 71911e8..a62984a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadPutGetSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadPutGetSelfTest.java
@@ -33,6 +33,7 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
@@ -163,6 +164,9 @@ public class GridCacheDhtPreloadPutGetSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPutGetNone1() throws Exception {
+        if (MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-10261");
+
         preloadMode = NONE;
         backups = 1;
 
@@ -173,6 +177,9 @@ public class GridCacheDhtPreloadPutGetSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPutGetNone2() throws Exception {
+        if (MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-10261");
+
         preloadMode = NONE;
         backups = 2;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
index e47d775..76b1424 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
@@ -212,7 +212,6 @@ public class GridCacheDhtPreloadSelfTest extends GridCommonAbstractTest {
      */
     private void checkActivePartitionTransfer(int keyCnt, int nodeCnt, boolean sameCoord, boolean shuffle)
         throws Exception {
-
         try {
             Ignite ignite1 = startGrid(0);
 
@@ -456,7 +455,6 @@ public class GridCacheDhtPreloadSelfTest extends GridCommonAbstractTest {
      */
     private void checkNodes(int keyCnt, int nodeCnt, boolean sameCoord, boolean shuffle)
         throws Exception {
-
         try {
             Ignite ignite1 = startGrid(0);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheClearDuringRebalanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheClearDuringRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheClearDuringRebalanceTest.java
index 8561c5c..5194020 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheClearDuringRebalanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheClearDuringRebalanceTest.java
@@ -28,6 +28,7 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
@@ -43,6 +44,14 @@ public class IgniteCacheClearDuringRebalanceTest extends GridCommonAbstractTest
     private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        if(MvccFeatureChecker.forcedMvcc())
+            fail("https://issues.apache.org/jira/browse/IGNITE-7952");
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         super.afterTest();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedBackupNodeFailureRecoveryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedBackupNodeFailureRecoveryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedBackupNodeFailureRecoveryTest.java
index 98520ab..6cecce1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedBackupNodeFailureRecoveryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedBackupNodeFailureRecoveryTest.java
@@ -44,7 +44,7 @@ import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
 /**
  */
 public class IgniteCachePartitionedBackupNodeFailureRecoveryTest extends IgniteCacheAbstractTest {
-    /** {@inheritDoc}*/
+  /** {@inheritDoc}*/
     @Override protected int gridCount() {
         return 3;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearClientHitTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearClientHitTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearClientHitTest.java
index fd378b7..6663a67 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearClientHitTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearClientHitTest.java
@@ -71,14 +71,11 @@ public class GridCacheNearClientHitTest extends GridCommonAbstractTest {
         CacheConfiguration<Object, Object> cfg = new CacheConfiguration<>();
 
         cfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
-
         cfg.setCacheMode(CacheMode.PARTITIONED);
-
         cfg.setBackups(1);
-
         cfg.setCopyOnRead(false);
-
         cfg.setName(CACHE_NAME);
+        cfg.setNearConfiguration(new NearCacheConfiguration<>());
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiGetSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiGetSelfTest.java
index 3e13ccd..4633161 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiGetSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiGetSelfTest.java
@@ -20,12 +20,13 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
@@ -35,9 +36,9 @@ import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
 import org.apache.ignite.transactions.TransactionOptimisticException;
-import org.apache.log4j.Level;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheRebalanceMode.NONE;
 import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
@@ -50,9 +51,6 @@ import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE;
  * Test getting the same value twice within the same transaction.
  */
 public class GridCacheNearMultiGetSelfTest extends GridCommonAbstractTest {
-    /** Cache debug flag. */
-    private static final boolean CACHE_DEBUG = false;
-
     /** Number of gets. */
     private static final int GET_CNT = 5;
 
@@ -62,40 +60,47 @@ public class GridCacheNearMultiGetSelfTest extends GridCommonAbstractTest {
     /** */
     private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
+    /** */
+    private CacheAtomicityMode atomicityMode;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
 
-        c.getTransactionConfiguration().setTxSerializableEnabled(true);
-
-        CacheConfiguration cc = defaultCacheConfiguration();
-
-        cc.setCacheMode(PARTITIONED);
-        cc.setBackups(1);
-        cc.setAtomicityMode(TRANSACTIONAL);
-
-        cc.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-
-        cc.setRebalanceMode(NONE);
-
         TcpDiscoverySpi spi = new TcpDiscoverySpi();
 
         spi.setIpFinder(ipFinder);
 
         c.setDiscoverySpi(spi);
 
-        c.setCacheConfiguration(cc);
+        return c;
+    }
 
-        if (CACHE_DEBUG)
-            resetLog4j(Level.DEBUG, false, GridCacheProcessor.class.getPackage().getName());
+    /**
+     * @return Cache configuration.
+     */
+    private CacheConfiguration cacheConfiguration() {
+        CacheConfiguration cc = defaultCacheConfiguration();
 
-        return c;
+        cc.setCacheMode(PARTITIONED);
+        cc.setBackups(1);
+        cc.setAtomicityMode(atomicityMode);
+        cc.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        cc.setRebalanceMode(NONE);
+        cc.setNearConfiguration(new NearCacheConfiguration());
+
+        return cc;
     }
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
-        for (int i = 0; i < GRID_CNT; i++)
-            startGrid(i);
+        startGridsMultiThreaded(GRID_CNT);
+    }
+
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        atomicityMode = TRANSACTIONAL;
     }
 
     /** {@inheritDoc} */
@@ -110,6 +115,8 @@ public class GridCacheNearMultiGetSelfTest extends GridCommonAbstractTest {
             assertEquals("Cache size mismatch for grid [igniteInstanceName=" + g.name() +
                     ", entrySet=" + entrySet(c) + ']', 0, c.size());
         }
+
+        grid(0).destroyCache(DEFAULT_CACHE_NAME);
     }
 
     /** @return {@code True} if debug enabled. */
@@ -213,6 +220,24 @@ public class GridCacheNearMultiGetSelfTest extends GridCommonAbstractTest {
         checkDoubleGet(PESSIMISTIC, SERIALIZABLE, true);
     }
 
+    /** @throws Exception If failed. */
+    public void testMvccPessimisticRepeatableReadNoPut() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-7187");
+
+        atomicityMode = TRANSACTIONAL_SNAPSHOT;
+
+        checkDoubleGet(PESSIMISTIC, REPEATABLE_READ, false);
+    }
+
+    /** @throws Exception If failed. */
+    public void testMvccPessimisticRepeatableReadWithPut() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-7187");
+
+        atomicityMode = TRANSACTIONAL_SNAPSHOT;
+
+        checkDoubleGet(PESSIMISTIC, REPEATABLE_READ, true);
+    }
+
     /**
      * @param concurrency Concurrency.
      * @param isolation Isolation.
@@ -222,7 +247,7 @@ public class GridCacheNearMultiGetSelfTest extends GridCommonAbstractTest {
     private void checkDoubleGet(TransactionConcurrency concurrency, TransactionIsolation isolation, boolean put)
         throws Exception {
         IgniteEx ignite = grid(0);
-        IgniteCache<Integer, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
+        IgniteCache<Integer, String> cache = ignite.getOrCreateCache(cacheConfiguration());
 
         Integer key = 1;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
index e83b14a..875284d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
@@ -56,6 +56,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.jetbrains.annotations.Nullable;
@@ -79,6 +80,15 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
     /** Cache store. */
     private static TestStore store = new TestStore();
 
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_STORE);
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
+        super.setUp();
+    }
+
     /** */
     private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOneNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOneNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOneNodeSelfTest.java
index 1595350..2b4b60c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOneNodeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOneNodeSelfTest.java
@@ -26,10 +26,12 @@ import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.store.CacheStoreAdapter;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 
@@ -46,6 +48,13 @@ public class GridCacheNearOneNodeSelfTest extends GridCommonAbstractTest {
     /** Cache store. */
     private static TestStore store = new TestStore();
 
+    /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
+        super.setUp();
+    }
+
     /**
      *
      */
@@ -80,8 +89,8 @@ public class GridCacheNearOneNodeSelfTest extends GridCommonAbstractTest {
         cacheCfg.setCacheMode(PARTITIONED);
         cacheCfg.setBackups(1);
         cacheCfg.setAtomicityMode(TRANSACTIONAL);
-
         cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        cacheCfg.setNearConfiguration(new NearCacheConfiguration());
 
         cacheCfg.setCacheStoreFactory(singletonFactory(store));
         cacheCfg.setReadThrough(true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearPartitionedClearSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearPartitionedClearSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearPartitionedClearSelfTest.java
index cbdc855..90a7406 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearPartitionedClearSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearPartitionedClearSelfTest.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -57,6 +58,13 @@ public class GridCacheNearPartitionedClearSelfTest extends GridCommonAbstractTes
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         startGrids(GRID_CNT);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReaderPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReaderPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReaderPreloadSelfTest.java
index e143260..316cc6e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReaderPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReaderPreloadSelfTest.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -62,6 +63,13 @@ public class GridCacheNearReaderPreloadSelfTest extends GridCommonAbstractTest {
     private IgniteCache<Integer, Integer> cache3;
 
     /** {@inheritDoc} */
+    @Override public void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         cache1 = null;
         cache2 = null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
index e30106c..88a757d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
@@ -48,6 +48,7 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
@@ -67,12 +68,19 @@ public class GridCacheNearReadersSelfTest extends GridCommonAbstractTest {
     private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** Grid counter. */
-    private static AtomicInteger cntr = new AtomicInteger(0);
+    private AtomicInteger cntr = new AtomicInteger(0);
 
     /** Test cache affinity. */
     private GridCacheModuloAffinityFunction aff = new GridCacheModuloAffinityFunction();
 
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java
index 91c9b22..73d5b5f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 
@@ -72,6 +73,11 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_STORE);
+
+        if (nearCacheConfiguration() != null)
+            MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
         super.beforeTestsStarted();
 
         stores = Collections.synchronizedList(new ArrayList<GridCacheTestStore>());

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEventSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEventSelfTest.java
index 1c79db7..54eb32d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEventSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEventSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheEventAbstractTest;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
@@ -31,6 +32,13 @@ import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
  */
 public class GridCachePartitionedEventSelfTest extends GridCacheEventAbstractTest {
     /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_EVENTS);
+
+        super.setUp();
+    }
+
+    /** {@inheritDoc} */
     @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception {
         CacheConfiguration cfg = defaultCacheConfiguration();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
index 96fb8f6..28afce1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
@@ -33,6 +33,7 @@ import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -51,6 +52,13 @@ public class GridCachePartitionedExplicitLockNodeFailureSelfTest extends GridCom
     public static final int GRID_CNT = 4;
 
     /** {@inheritDoc} */
+    @Override public void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.ENTRY_LOCK);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLoadCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLoadCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLoadCacheSelfTest.java
index 9474950..2ad8b50 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLoadCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLoadCacheSelfTest.java
@@ -29,6 +29,7 @@ import org.apache.ignite.lang.IgniteBiInClosure;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
@@ -49,6 +50,13 @@ public class GridCachePartitionedLoadCacheSelfTest extends GridCommonAbstractTes
     private static final int PUT_CNT = 100;
 
     /** {@inheritDoc} */
+    @Override public void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_STORE);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxMultiThreadedSelfTest.java
new file mode 100644
index 0000000..184132e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxMultiThreadedSelfTest.java
@@ -0,0 +1,101 @@
+/*
+ * 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.internal.processors.cache.distributed.near;
+
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.processors.cache.IgniteMvccTxMultiThreadedAbstractTest;
+import org.apache.ignite.testframework.MvccFeatureChecker;
+
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ * Tests for partitioned cache transactions.
+ */
+public class GridCachePartitionedMvccTxMultiThreadedSelfTest extends IgniteMvccTxMultiThreadedAbstractTest {
+    @Override protected void beforeTestsStarted() throws Exception {
+        if (nearEnabled())
+            MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg = defaultCacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+
+        ccfg.setEvictionPolicy(null);
+
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+
+        ccfg.setNearConfiguration(nearEnabled() ? new NearCacheConfiguration() : null);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /**
+     * @return {@code True} if near cache is enabled.
+     */
+    protected boolean nearEnabled() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int keyCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int maxKeyValue() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int threadCount() {
+        return 5;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int iterations() {
+        return 1000;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean isTestDebug() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean printMemoryStats() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxSingleThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxSingleThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxSingleThreadedSelfTest.java
new file mode 100644
index 0000000..1404839
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxSingleThreadedSelfTest.java
@@ -0,0 +1,84 @@
+/*
+ * 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.internal.processors.cache.distributed.near;
+
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.cache.IgniteMvccTxSingleThreadedAbstractTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheRebalanceMode.NONE;
+
+/**
+ * Tests for partitioned cache transactions.
+ */
+public class GridCachePartitionedMvccTxSingleThreadedSelfTest extends IgniteMvccTxSingleThreadedAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> ccfg = defaultCacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setAtomicityMode(TRANSACTIONAL_SNAPSHOT);
+
+        ccfg.setNearConfiguration(null);
+        ccfg.setEvictionPolicy(null);
+
+        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_ASYNC);
+
+        ccfg.setRebalanceMode(NONE);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 4;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int keyCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int maxKeyValue() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int iterations() {
+        return 3000;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean isTestDebug() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean printMemoryStats() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxTimeoutSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxTimeoutSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxTimeoutSelfTest.java
new file mode 100644
index 0000000..aad95aa
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxTimeoutSelfTest.java
@@ -0,0 +1,47 @@
+/*
+ * 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.internal.processors.cache.distributed.near;
+
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.cache.distributed.IgniteMvccTxTimeoutAbstractTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+
+/**
+ * Simple cache test.
+ */
+public class GridCachePartitionedMvccTxTimeoutSelfTest extends IgniteMvccTxTimeoutAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration cc = defaultCacheConfiguration();
+
+        cc.setCacheMode(PARTITIONED);
+        cc.setBackups(1);
+        cc.setAtomicityMode(TRANSACTIONAL);
+
+        //cacheCfg.setPreloadMode(NONE);
+
+        c.setCacheConfiguration(cc);
+
+        return c;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedPreloadLifecycleSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedPreloadLifecycleSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedPreloadLifecycleSelfTest.java
index da802e3..7f25cba 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedPreloadLifecycleSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedPreloadLifecycleSelfTest.java
@@ -56,7 +56,6 @@ public class GridCachePartitionedPreloadLifecycleSelfTest extends GridCachePrelo
         cc1.setRebalanceMode(preloadMode);
         cc1.setEvictionPolicy(null);
         cc1.setCacheStoreFactory(null);
-        cc1.setEvictionPolicy(null);
 
         // Identical configuration.
         CacheConfiguration cc2 = new CacheConfiguration(cc1);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxTimeoutSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxTimeoutSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxTimeoutSelfTest.java
index 202476e..1a88ae0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxTimeoutSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxTimeoutSelfTest.java
@@ -19,11 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.TransactionConfiguration;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteTxTimeoutAbstractTest;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
@@ -32,27 +28,10 @@ import static org.apache.ignite.cache.CacheMode.PARTITIONED;
  * Simple cache test.
  */
 public class GridCachePartitionedTxTimeoutSelfTest extends IgniteTxTimeoutAbstractTest {
-    /** Transaction timeout. */
-    private static final long TIMEOUT = 50;
-
-    /** */
-    private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
 
-        TransactionConfiguration txCfg = c.getTransactionConfiguration();
-
-        txCfg.setTxSerializableEnabled(true);
-        txCfg.setDefaultTxTimeout(TIMEOUT);
-
-        TcpDiscoverySpi spi = new TcpDiscoverySpi();
-
-        spi.setIpFinder(ipFinder);
-
-        c.setDiscoverySpi(spi);
-
         CacheConfiguration cc = defaultCacheConfiguration();
 
         cc.setCacheMode(PARTITIONED);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheStoreUpdateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheStoreUpdateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheStoreUpdateTest.java
index bf8ad78..02a9223 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheStoreUpdateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheStoreUpdateTest.java
@@ -39,6 +39,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
@@ -60,6 +61,14 @@ public class GridNearCacheStoreUpdateTest extends GridCommonAbstractTest {
     private IgniteCache<String, String> cache;
 
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.CACHE_STORE);
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(final String gridName) throws Exception {
         final IgniteConfiguration cfg = super.getConfiguration(gridName);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NearCacheSyncUpdateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NearCacheSyncUpdateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NearCacheSyncUpdateTest.java
index d253013..e1ca01f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NearCacheSyncUpdateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NearCacheSyncUpdateTest.java
@@ -34,6 +34,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 
@@ -60,6 +61,13 @@ public class NearCacheSyncUpdateTest extends GridCommonAbstractTest {
         startGridsMultiThreaded(3);
     }
 
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
     /**
      * @throws Exception If failed.
      */
@@ -75,6 +83,15 @@ public class NearCacheSyncUpdateTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testNearCacheSyncUpdateMvccTx() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-7187");
+
+        nearCacheSyncUpdateTx(TRANSACTIONAL_SNAPSHOT);
+    }
+
+    /**
      * @param atomicityMode Atomicity mode.
      * @throws Exception If failed.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NoneRebalanceModeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NoneRebalanceModeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NoneRebalanceModeSelfTest.java
index c1123cb..109dbff 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NoneRebalanceModeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NoneRebalanceModeSelfTest.java
@@ -45,9 +45,18 @@ public class NoneRebalanceModeSelfTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
         startGrid(0);
     }
 
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
     /**
      * @throws Exception If failed.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedTxTimeoutSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedTxTimeoutSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedTxTimeoutSelfTest.java
index a63a302..9a748ba 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedTxTimeoutSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedTxTimeoutSelfTest.java
@@ -20,9 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.replicated;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteTxTimeoutAbstractTest;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 
@@ -30,31 +27,16 @@ import static org.apache.ignite.cache.CacheMode.REPLICATED;
  * Simple cache test.
  */
 public class GridCacheReplicatedTxTimeoutSelfTest extends IgniteTxTimeoutAbstractTest {
-    /** Transaction timeout. */
-    private static final long TIMEOUT = 50;
-
-    /** */
-    private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
 
-        c.getTransactionConfiguration().setDefaultTxTimeout(TIMEOUT);
-        c.getTransactionConfiguration().setTxSerializableEnabled(true);
-
         CacheConfiguration cc = defaultCacheConfiguration();
 
         cc.setCacheMode(REPLICATED);
 
         c.setCacheConfiguration(cc);
 
-        TcpDiscoverySpi spi = new TcpDiscoverySpi();
-
-        spi.setIpFinder(ipFinder);
-
-        c.setDiscoverySpi(spi);
-
         return c;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalBasicApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalBasicApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalBasicApiSelfTest.java
index d35590f..91df599 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalBasicApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalBasicApiSelfTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.local;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheBasicApiAbstractTest;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 
@@ -28,6 +29,13 @@ import static org.apache.ignite.cache.CacheMode.LOCAL;
  */
 public class GridCacheLocalBasicApiSelfTest extends GridCacheBasicApiAbstractTest {
     /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.LOCAL_CACHE);
+
+        super.setUp();
+    }
+
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalBasicStoreSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalBasicStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalBasicStoreSelfTest.java
index 183b456..5d6492b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalBasicStoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalBasicStoreSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.local;
 
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.internal.processors.cache.GridCacheBasicStoreAbstractTest;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 
@@ -27,6 +28,13 @@ import static org.apache.ignite.cache.CacheMode.LOCAL;
  */
 public class GridCacheLocalBasicStoreSelfTest extends GridCacheBasicStoreAbstractTest {
     /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.LOCAL_CACHE);
+
+        super.setUp();
+    }
+
+    /** {@inheritDoc} */
     @Override protected CacheMode cacheMode() {
         return LOCAL;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalEventSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalEventSelfTest.java
index 0bf01c2..c0b6071 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalEventSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalEventSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.local;
 
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheEventAbstractTest;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 
@@ -27,6 +28,14 @@ import static org.apache.ignite.cache.CacheMode.LOCAL;
  */
 public class GridCacheLocalEventSelfTest extends GridCacheEventAbstractTest {
     /** {@inheritDoc} */
+    @Override public void beforeTestsStarted() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.LOCAL_CACHE);
+
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
     @Override protected CacheMode cacheMode() {
         return LOCAL;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalEvictionEventSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalEvictionEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalEvictionEventSelfTest.java
index b2d98c6..7406689 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalEvictionEventSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalEvictionEventSelfTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.local;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.internal.processors.cache.GridCacheEvictionEventAbstractTest;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 
@@ -28,6 +29,13 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
  */
 public class GridCacheLocalEvictionEventSelfTest extends GridCacheEvictionEventAbstractTest {
     /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.LOCAL_CACHE);
+
+        super.setUp();
+    }
+
+    /** {@inheritDoc} */
     @Override protected CacheMode cacheMode() {
         return CacheMode.LOCAL;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7577c877/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalGetAndTransformStoreSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalGetAndTransformStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalGetAndTransformStoreSelfTest.java
index 19b4b61..9752b7e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalGetAndTransformStoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalGetAndTransformStoreSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.local;
 
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.internal.processors.cache.GridCacheGetAndTransformStoreAbstractTest;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 
@@ -27,6 +28,13 @@ import static org.apache.ignite.cache.CacheMode.LOCAL;
  */
 public class GridCacheLocalGetAndTransformStoreSelfTest extends GridCacheGetAndTransformStoreAbstractTest {
     /** {@inheritDoc} */
+    @Override public void setUp() throws Exception {
+        MvccFeatureChecker.failIfNotSupported(MvccFeatureChecker.Feature.LOCAL_CACHE);
+
+        super.setUp();
+    }
+
+    /** {@inheritDoc} */
     @Override protected CacheMode cacheMode() {
         return LOCAL;
     }


[19/50] [abbrv] ignite git commit: IGNITE-10330: Disk page compression. - Fixes #5200.

Posted by ag...@apache.org.
IGNITE-10330: Disk page compression. - Fixes #5200.

Signed-off-by: Sergi Vladykin <se...@gmail.com>


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

Branch: refs/heads/ignite-9720
Commit: e8eeea375584bb31a8a9f60e43c8206cd4753d64
Parents: 9137af7
Author: Sergi Vladykin <se...@gmail.com>
Authored: Mon Nov 26 15:37:58 2018 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Mon Nov 26 15:37:58 2018 +0300

----------------------------------------------------------------------
 modules/compress/licenses/Snappy-LICENSE.txt    |   29 +
 .../compress/licenses/Snappy-java-LICENSE.txt   |  202 ++
 modules/compress/licenses/Zstd-LICENSE.txt      |   30 +
 modules/compress/licenses/Zstd-jni-LICENSE.txt  |   26 +
 modules/compress/licenses/jnr-ffi-LICENSE.txt   |   25 +
 modules/compress/licenses/jnr-posix-LICENSE.txt | 2486 ++++++++++++++++++
 modules/compress/licenses/lz4-LICENSE.txt       |   11 +
 modules/compress/licenses/lz4-java-LICENSE.txt  |  202 ++
 modules/compress/pom.xml                        |  113 +
 .../compress/CompressionProcessorImpl.java      |  403 +++
 .../compress/NativeFileSystemLinux.java         |  142 +
 .../compress/NativeFileSystemPosix.java         |   54 +
 .../compress/CompressionProcessorTest.java      | 1021 +++++++
 ...DiskPageCompressionIntegrationAsyncTest.java |   30 +
 .../DiskPageCompressionIntegrationTest.java     |  468 ++++
 .../compress/FileSystemUtilsTest.java           |  193 ++
 .../IgnitePdsCompressionTestSuite.java          |   56 +
 .../IgnitePdsCompressionTestSuite2.java         |   38 +
 .../org/apache/ignite/DataStorageMetrics.java   |   18 +
 .../apache/ignite/IgniteSystemProperties.java   |   44 +
 .../configuration/CacheConfiguration.java       |   60 +
 .../configuration/DataStorageConfiguration.java |   16 +-
 .../configuration/DiskPageCompression.java      |   38 +
 .../ignite/internal/GridKernalContext.java      |    6 +
 .../ignite/internal/GridKernalContextImpl.java  |   12 +
 .../ignite/internal/IgniteComponentType.java    |    7 +
 .../apache/ignite/internal/IgniteKernal.java    |    2 +
 .../internal/pagemem/store/PageStore.java       |   34 +-
 .../cache/CacheCompressionManager.java          |   95 +
 .../cache/CacheGroupMetricsMXBeanImpl.java      |   23 +-
 .../processors/cache/GridCacheContext.java      |   13 +
 .../processors/cache/GridCacheProcessor.java    |    3 +
 .../persistence/DataStorageMetricsImpl.java     |   22 +-
 .../persistence/DataStorageMetricsSnapshot.java |   18 +
 .../GridCacheDatabaseSharedManager.java         |   89 +-
 .../cache/persistence/file/AbstractFileIO.java  |    2 +-
 .../cache/persistence/file/AsyncFileIO.java     |   38 +-
 .../cache/persistence/file/EncryptedFileIO.java |   16 +-
 .../cache/persistence/file/FileIO.java          |   19 +
 .../cache/persistence/file/FileIODecorator.java |   17 +-
 .../cache/persistence/file/FilePageStore.java   |   72 +-
 .../persistence/file/FilePageStoreManager.java  |   65 +-
 .../cache/persistence/file/FilePageStoreV2.java |   21 +
 .../persistence/file/RandomAccessFileIO.java    |   35 +
 .../cache/persistence/file/UnzipFileIO.java     |   15 +
 .../persistence/pagemem/PageMemoryImpl.java     |   20 +-
 .../persistence/tree/io/AbstractDataPageIO.java |   79 +-
 .../cache/persistence/tree/io/BPlusIO.java      |   32 +-
 .../persistence/tree/io/CompactablePageIO.java  |   43 +
 .../persistence/tree/io/DataPagePayload.java    |   16 +
 .../cache/persistence/tree/io/PageIO.java       |  103 +-
 .../wal/reader/StandaloneGridKernalContext.java |    6 +
 .../compress/CompressionProcessor.java          |  173 ++
 .../processors/compress/FileSystemUtils.java    |  129 +
 .../processors/compress/NativeFileSystem.java   |   50 +
 .../apache/ignite/internal/util/GridUnsafe.java |    1 +
 .../ignite/internal/util/IgniteUtils.java       |   24 +-
 .../ignite/mxbean/CacheGroupMetricsMXBean.java  |   12 +
 .../ignite/mxbean/DataStorageMetricsMXBean.java |    8 +
 .../IgniteClusterActivateDeactivateTest.java    |    2 +-
 .../cache/persistence/DummyPageIO.java          |   21 +-
 .../IgniteDataStorageMetricsSelfTest.java       |   17 +-
 .../persistence/IgnitePdsDynamicCacheTest.java  |    3 +-
 .../IgnitePdsExchangeDuringCheckpointTest.java  |    1 +
 .../persistence/IgnitePdsPageSizesTest.java     |    8 +
 ...lWalModeChangeDuringRebalancingSelfTest.java |   15 +
 .../db/IgnitePdsCacheRestoreTest.java           |    8 +-
 .../db/IgnitePdsDataRegionMetricsTest.java      |   10 +-
 .../db/IgnitePdsReserveWalSegmentsTest.java     |    2 -
 ...gniteCheckpointDirtyPagesForLowLoadTest.java |    1 -
 ...faultPageSizeBackwardsCompatibilityTest.java |   19 +-
 .../persistence/db/wal/WalCompactionTest.java   |   26 +-
 .../db/wal/WalDeletionArchiveAbstractTest.java  |    1 -
 .../database/IgniteDbAbstractTest.java          |    2 -
 .../loadtests/hashmap/GridCacheTestContext.java |    2 +
 modules/direct-io/pom.xml                       |   17 +-
 .../file/AlignedBuffersDirectFileIO.java        |  112 +-
 .../file/AlignedBuffersDirectFileIOFactory.java |   16 +-
 .../persistence/file/IgniteNativeIoLib.java     |    2 +-
 ...kPageCompressionIntegrationDirectIOTest.java |   31 +
 .../persistence/file/IgniteFileIOTest.java      |   15 +
 .../testsuites/IgnitePdsNativeIoTestSuite2.java |    4 +
 parent/pom.xml                                  |    5 +
 pom.xml                                         |    1 +
 84 files changed, 7208 insertions(+), 158 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/compress/licenses/Snappy-LICENSE.txt
----------------------------------------------------------------------
diff --git a/modules/compress/licenses/Snappy-LICENSE.txt b/modules/compress/licenses/Snappy-LICENSE.txt
new file mode 100644
index 0000000..530d2f2
--- /dev/null
+++ b/modules/compress/licenses/Snappy-LICENSE.txt
@@ -0,0 +1,29 @@
+Copyright 2011, Google Inc.
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+    * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) 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
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/compress/licenses/Snappy-java-LICENSE.txt
----------------------------------------------------------------------
diff --git a/modules/compress/licenses/Snappy-java-LICENSE.txt b/modules/compress/licenses/Snappy-java-LICENSE.txt
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/modules/compress/licenses/Snappy-java-LICENSE.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/ignite/blob/e8eeea37/modules/compress/licenses/Zstd-LICENSE.txt
----------------------------------------------------------------------
diff --git a/modules/compress/licenses/Zstd-LICENSE.txt b/modules/compress/licenses/Zstd-LICENSE.txt
new file mode 100644
index 0000000..a793a80
--- /dev/null
+++ b/modules/compress/licenses/Zstd-LICENSE.txt
@@ -0,0 +1,30 @@
+BSD License
+
+For Zstandard software
+
+Copyright (c) 2016-present, Facebook, Inc. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+ * Redistributions of source code must retain the above copyright notice, this
+   list of conditions and the following disclaimer.
+
+ * Redistributions in binary form must reproduce the above copyright notice,
+   this list of conditions and the following disclaimer in the documentation
+   and/or other materials provided with the distribution.
+
+ * Neither the name Facebook nor the names of its contributors may be used to
+   endorse or promote products derived from this software without specific
+   prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) 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 OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/compress/licenses/Zstd-jni-LICENSE.txt
----------------------------------------------------------------------
diff --git a/modules/compress/licenses/Zstd-jni-LICENSE.txt b/modules/compress/licenses/Zstd-jni-LICENSE.txt
new file mode 100644
index 0000000..db1103a
--- /dev/null
+++ b/modules/compress/licenses/Zstd-jni-LICENSE.txt
@@ -0,0 +1,26 @@
+Zstd-jni: JNI bindings to Zstd Library
+
+Copyright (c) 2015-2018, Luben Karavelov/ All rights reserved.
+
+BSD License
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice, this
+  list of conditions and the following disclaimer in the documentation and/or
+  other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) 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 OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/compress/licenses/jnr-ffi-LICENSE.txt
----------------------------------------------------------------------
diff --git a/modules/compress/licenses/jnr-ffi-LICENSE.txt b/modules/compress/licenses/jnr-ffi-LICENSE.txt
new file mode 100644
index 0000000..a3a8343
--- /dev/null
+++ b/modules/compress/licenses/jnr-ffi-LICENSE.txt
@@ -0,0 +1,25 @@
+  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.
+
+
+  Alternatively, you can redistribute it and/or modify it under
+  the terms of the GNU Lesser General Public License as published by
+  the Free Software Foundation, either version 3 of the License, or
+  (at your option) any later version.
+
+  This code is distributed in the hope that it will be useful, but WITHOUT
+  ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
+  FITNESS FOR A PARTICULAR PURPOSE.  See the GNU Lesser General Public License
+  version 3 for more details.
+
+  You should have received a copy of the GNU Lesser General Public License
+  version 3 along with this work.  If not, see <http://www.gnu.org/licenses/>.


[14/50] [abbrv] ignite git commit: IGNITE-10330: Disk page compression. - Fixes #5200.

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/direct-io/src/test/java/org/apache/ignite/internal/processors/cache/persistence/file/IgniteFileIOTest.java
----------------------------------------------------------------------
diff --git a/modules/direct-io/src/test/java/org/apache/ignite/internal/processors/cache/persistence/file/IgniteFileIOTest.java b/modules/direct-io/src/test/java/org/apache/ignite/internal/processors/cache/persistence/file/IgniteFileIOTest.java
index 9620eb0..640932dc 100644
--- a/modules/direct-io/src/test/java/org/apache/ignite/internal/processors/cache/persistence/file/IgniteFileIOTest.java
+++ b/modules/direct-io/src/test/java/org/apache/ignite/internal/processors/cache/persistence/file/IgniteFileIOTest.java
@@ -55,6 +55,21 @@ public class IgniteFileIOTest extends TestCase {
         }
 
         /** {@inheritDoc} */
+        @Override public int getFileSystemBlockSize() {
+            return -1;
+        }
+
+        /** {@inheritDoc} */
+        @Override public long getSparseSize() {
+            return -1;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int punchHole(long position, int len) {
+            throw new UnsupportedOperationException();
+        }
+
+        /** {@inheritDoc} */
         @Override public long position() throws IOException {
             return position;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/modules/direct-io/src/test/java/org/apache/ignite/testsuites/IgnitePdsNativeIoTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/direct-io/src/test/java/org/apache/ignite/testsuites/IgnitePdsNativeIoTestSuite2.java b/modules/direct-io/src/test/java/org/apache/ignite/testsuites/IgnitePdsNativeIoTestSuite2.java
index 2ed7450..dab99c9 100644
--- a/modules/direct-io/src/test/java/org/apache/ignite/testsuites/IgnitePdsNativeIoTestSuite2.java
+++ b/modules/direct-io/src/test/java/org/apache/ignite/testsuites/IgnitePdsNativeIoTestSuite2.java
@@ -19,6 +19,7 @@ package org.apache.ignite.testsuites;
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.cache.persistence.IgniteNativeIoLocalWalModeChangeDuringRebalancingSelfTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgniteNativeIoPdsRecoveryAfterFileCorruptionTest;
+import org.apache.ignite.internal.processors.cache.persistence.DiskPageCompressionIntegrationDirectIOTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteNativeIoWalFlushFsyncSelfTest;
 
 /**
@@ -34,6 +35,9 @@ public class IgnitePdsNativeIoTestSuite2 extends TestSuite {
 
         IgnitePdsTestSuite2.addRealPageStoreTests(suite);
 
+        // Direct IO + Page compression.
+        suite.addTestSuite(DiskPageCompressionIntegrationDirectIOTest.class);
+
         //Integrity test with reduced count of pages.
         suite.addTestSuite(IgniteNativeIoPdsRecoveryAfterFileCorruptionTest.class);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 14f970e..23e0a0d 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -84,6 +84,8 @@
         <jetty.version>9.4.11.v20180605</jetty.version>
         <jmh.version>1.13</jmh.version>
         <jms.spec.version>1.1.1</jms.spec.version>
+        <jna.version>4.5.2</jna.version>
+        <jnr.posix.version>3.0.46</jnr.posix.version>
         <jsch.bundle.version>0.1.54_1</jsch.bundle.version>
         <jsch.version>0.1.54</jsch.version>
         <jsonlib.bundle.version>2.4_1</jsonlib.bundle.version>
@@ -94,6 +96,7 @@
         <log4j.version>2.11.0</log4j.version>
         <lucene.bundle.version>7.4.0_1</lucene.bundle.version>
         <lucene.version>7.4.0</lucene.version>
+        <lz4.version>1.5.0</lz4.version>
         <maven.bundle.plugin.version>3.5.0</maven.bundle.plugin.version>
         <mockito.version>1.10.19</mockito.version>
         <mysql.connector.version>5.1.39</mysql.connector.version>
@@ -110,6 +113,7 @@
         <scala.test.version>2.2.6</scala.test.version>
         <slf4j.version>1.7.7</slf4j.version>
         <slf4j16.version>1.6.4</slf4j16.version>
+        <snappy.version>1.1.7.2</snappy.version>
         <spark.hadoop.version>2.6.5</spark.hadoop.version>
         <spark.version>2.3.0</spark.version>
         <spring.data.version>1.13.14.RELEASE</spring.data.version> <!-- don't forget to update spring version -->
@@ -125,6 +129,7 @@
         <yardstick.version>0.8.3</yardstick.version>
         <zkclient.version>0.5</zkclient.version>
         <zookeeper.version>3.4.13</zookeeper.version>
+        <zstd.version>1.3.7-2</zstd.version>
 
         <!-- OSGI Manifest generation default property values -->
         <osgi.import.package>*</osgi.import.package>

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8eeea37/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6de435a..54756bb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -45,6 +45,7 @@
         <module>modules/apache-license-gen</module>
         <module>modules/tools</module>
         <module>modules/core</module>
+        <module>modules/compress</module>
         <module>modules/dev-utils</module>
         <module>modules/direct-io</module>
         <module>modules/hadoop</module>


[43/50] [abbrv] ignite git commit: IGNITE-8542: [ML] Add OneVsRest Trainer to handle cases with multiple class labels in dataset.

Posted by ag...@apache.org.
IGNITE-8542: [ML] Add OneVsRest Trainer to handle cases with
multiple class labels in dataset.

This closes #5512


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

Branch: refs/heads/ignite-9720
Commit: c3fd4a930cc1a76b4d1fbccc6d764bdfe88da941
Parents: 3885f3f
Author: zaleslaw <za...@gmail.com>
Authored: Wed Nov 28 01:45:11 2018 +0300
Committer: Yury Babak <yb...@gridgain.com>
Committed: Wed Nov 28 01:45:11 2018 +0300

----------------------------------------------------------------------
 .../ignite/ml/multiclass/MultiClassModel.java   | 115 +++++++++++++++
 .../ignite/ml/multiclass/OneVsRestTrainer.java  | 147 +++++++++++++++++++
 .../org/apache/ignite/ml/IgniteMLTestSuite.java |   4 +-
 .../ml/multiclass/MultiClassTestSuite.java      |  32 ++++
 .../ml/multiclass/OneVsRestTrainerTest.java     | 126 ++++++++++++++++
 5 files changed, 423 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c3fd4a93/modules/ml/src/main/java/org/apache/ignite/ml/multiclass/MultiClassModel.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/multiclass/MultiClassModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/multiclass/MultiClassModel.java
new file mode 100644
index 0000000..8520aa9
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/multiclass/MultiClassModel.java
@@ -0,0 +1,115 @@
+/*
+ * 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.ml.multiclass;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.TreeMap;
+import org.apache.ignite.ml.Exportable;
+import org.apache.ignite.ml.Exporter;
+import org.apache.ignite.ml.Model;
+import org.apache.ignite.ml.math.primitives.vector.Vector;
+
+/** Base class for multi-classification model for set of classifiers. */
+public class MultiClassModel<M extends Model<Vector, Double>> implements Model<Vector, Double>, Exportable<MultiClassModel>, Serializable {
+    /** */
+    private static final long serialVersionUID = -114986533359917L;
+
+    /** List of models associated with each class. */
+    private Map<Double, M> models;
+
+    /** */
+    public MultiClassModel() {
+        this.models = new HashMap<>();
+    }
+
+    /**
+     * Adds a specific binary classifier to the bunch of same classifiers.
+     *
+     * @param clsLb The class label for the added model.
+     * @param mdl The model.
+     */
+    public void add(double clsLb, M mdl) {
+        models.put(clsLb, mdl);
+    }
+
+    /**
+     * @param clsLb Class label.
+     * @return model for class label if it exists.
+     */
+    public Optional<M> getModel(Double clsLb) {
+        return Optional.ofNullable(models.get(clsLb));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Double apply(Vector input) {
+        TreeMap<Double, Double> maxMargins = new TreeMap<>();
+
+        models.forEach((k, v) -> maxMargins.put(v.apply(input), k));
+
+        // returns value the most closest to 1
+        return maxMargins.lastEntry().getValue();
+    }
+
+    /** {@inheritDoc} */
+    @Override public <P> void saveModel(Exporter<MultiClassModel, P> exporter, P path) {
+        exporter.save(this, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        MultiClassModel mdl = (MultiClassModel)o;
+
+        return Objects.equals(models, mdl.models);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return Objects.hash(models);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        StringBuilder wholeStr = new StringBuilder();
+
+        models.forEach((clsLb, mdl) ->
+            wholeStr
+                .append("The class with label ")
+                .append(clsLb)
+                .append(" has classifier: ")
+                .append(mdl.toString())
+                .append(System.lineSeparator())
+        );
+
+        return wholeStr.toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString(boolean pretty) {
+        return toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c3fd4a93/modules/ml/src/main/java/org/apache/ignite/ml/multiclass/OneVsRestTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/multiclass/OneVsRestTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/multiclass/OneVsRestTrainer.java
new file mode 100644
index 0000000..7426506
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/multiclass/OneVsRestTrainer.java
@@ -0,0 +1,147 @@
+/*
+ * 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.ml.multiclass;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.ignite.ml.Model;
+import org.apache.ignite.ml.dataset.Dataset;
+import org.apache.ignite.ml.dataset.DatasetBuilder;
+import org.apache.ignite.ml.dataset.PartitionDataBuilder;
+import org.apache.ignite.ml.dataset.primitive.context.EmptyContext;
+import org.apache.ignite.ml.math.functions.IgniteBiFunction;
+import org.apache.ignite.ml.math.primitives.vector.Vector;
+import org.apache.ignite.ml.structures.partition.LabelPartitionDataBuilderOnHeap;
+import org.apache.ignite.ml.structures.partition.LabelPartitionDataOnHeap;
+import org.apache.ignite.ml.trainers.SingleLabelDatasetTrainer;
+
+/**
+ * This is a common heuristic trainer for multi-class labeled models.
+ *
+ * NOTE: The current implementation suffers from unbalanced training over the dataset due to unweighted approach
+ * during the process of reassign labels from all range of labels to 0,1.
+ */
+public class OneVsRestTrainer<M extends Model<Vector, Double>>
+    extends SingleLabelDatasetTrainer<MultiClassModel<M>> {
+    /** The common binary classifier with all hyper-parameters to spread them for all separate trainings . */
+    private SingleLabelDatasetTrainer<M> classifier;
+
+    /** */
+    public OneVsRestTrainer(SingleLabelDatasetTrainer<M> classifier) {
+        this.classifier = classifier;
+    }
+
+    /**
+     * Trains model based on the specified data.
+     *
+     * @param datasetBuilder Dataset builder.
+     * @param featureExtractor Feature extractor.
+     * @param lbExtractor Label extractor.
+     * @return Model.
+     */
+    @Override public <K, V> MultiClassModel<M> fit(DatasetBuilder<K, V> datasetBuilder,
+        IgniteBiFunction<K, V, Vector> featureExtractor,
+        IgniteBiFunction<K, V, Double> lbExtractor) {
+
+        return updateModel(null, datasetBuilder, featureExtractor, lbExtractor);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> MultiClassModel<M> updateModel(MultiClassModel<M> newMdl,
+        DatasetBuilder<K, V> datasetBuilder, IgniteBiFunction<K, V, Vector> featureExtractor,
+        IgniteBiFunction<K, V, Double> lbExtractor) {
+
+        List<Double> classes = extractClassLabels(datasetBuilder, lbExtractor);
+
+        if (classes.isEmpty())
+            return getLastTrainedModelOrThrowEmptyDatasetException(newMdl);
+
+        MultiClassModel<M> multiClsMdl = new MultiClassModel<>();
+
+        classes.forEach(clsLb -> {
+            IgniteBiFunction<K, V, Double> lbTransformer = (k, v) -> {
+                Double lb = lbExtractor.apply(k, v);
+
+                if (lb.equals(clsLb))
+                    return 1.0;
+                else
+                    return 0.0;
+            };
+
+            M mdl = Optional.ofNullable(newMdl)
+                .flatMap(multiClassModel -> multiClassModel.getModel(clsLb))
+                .map(learnedModel -> classifier.update(learnedModel, datasetBuilder, featureExtractor, lbTransformer))
+                .orElseGet(() -> classifier.fit(datasetBuilder, featureExtractor, lbTransformer));
+
+            multiClsMdl.add(clsLb, mdl);
+        });
+
+        return multiClsMdl;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean checkState(MultiClassModel<M> mdl) {
+        return true;
+    }
+
+    /** Iterates among dataset and collects class labels. */
+    private <K, V> List<Double> extractClassLabels(DatasetBuilder<K, V> datasetBuilder,
+        IgniteBiFunction<K, V, Double> lbExtractor) {
+        assert datasetBuilder != null;
+
+        PartitionDataBuilder<K, V, EmptyContext, LabelPartitionDataOnHeap> partDataBuilder = new LabelPartitionDataBuilderOnHeap<>(lbExtractor);
+
+        List<Double> res = new ArrayList<>();
+
+        try (Dataset<EmptyContext, LabelPartitionDataOnHeap> dataset = datasetBuilder.build(
+            (upstream, upstreamSize) -> new EmptyContext(),
+            partDataBuilder
+        )) {
+            final Set<Double> clsLabels = dataset.compute(data -> {
+                final Set<Double> locClsLabels = new HashSet<>();
+
+                final double[] lbs = data.getY();
+
+                for (double lb : lbs)
+                    locClsLabels.add(lb);
+
+                return locClsLabels;
+            }, (a, b) -> {
+                if (a == null)
+                    return b == null ? new HashSet<>() : b;
+                if (b == null)
+                    return a;
+                return Stream.of(a, b).flatMap(Collection::stream).collect(Collectors.toSet());
+            });
+
+            if (clsLabels != null)
+                res.addAll(clsLabels);
+
+        }
+        catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+        return res;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c3fd4a93/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java
index f9645d8..78d6659 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java
@@ -26,6 +26,7 @@ import org.apache.ignite.ml.genetic.GAGridTestSuite;
 import org.apache.ignite.ml.inference.InferenceTestSuite;
 import org.apache.ignite.ml.knn.KNNTestSuite;
 import org.apache.ignite.ml.math.MathImplMainTestSuite;
+import org.apache.ignite.ml.multiclass.MultiClassTestSuite;
 import org.apache.ignite.ml.nn.MLPTestSuite;
 import org.apache.ignite.ml.pipeline.PipelineTestSuite;
 import org.apache.ignite.ml.preprocessing.PreprocessingTestSuite;
@@ -61,7 +62,8 @@ import org.junit.runners.Suite;
     StructuresTestSuite.class,
     CommonTestSuite.class,
     InferenceTestSuite.class,
-    BaggingTest.class
+    BaggingTest.class,
+    MultiClassTestSuite.class
 })
 public class IgniteMLTestSuite {
     // No-op.

http://git-wip-us.apache.org/repos/asf/ignite/blob/c3fd4a93/modules/ml/src/test/java/org/apache/ignite/ml/multiclass/MultiClassTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/multiclass/MultiClassTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/multiclass/MultiClassTestSuite.java
new file mode 100644
index 0000000..551597f
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/multiclass/MultiClassTestSuite.java
@@ -0,0 +1,32 @@
+/*
+ * 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.ml.multiclass;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+
+/**
+ * Test suite for multilayer perceptrons.
+ */
+@RunWith(Suite.class)
+@Suite.SuiteClasses({
+    OneVsRestTrainerTest.class
+})
+public class MultiClassTestSuite {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c3fd4a93/modules/ml/src/test/java/org/apache/ignite/ml/multiclass/OneVsRestTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/multiclass/OneVsRestTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/multiclass/OneVsRestTrainerTest.java
new file mode 100644
index 0000000..9842d92
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/multiclass/OneVsRestTrainerTest.java
@@ -0,0 +1,126 @@
+/*
+ * 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.ml.multiclass;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.ml.TestUtils;
+import org.apache.ignite.ml.common.TrainerTest;
+import org.apache.ignite.ml.math.primitives.vector.Vector;
+import org.apache.ignite.ml.math.primitives.vector.VectorUtils;
+import org.apache.ignite.ml.nn.UpdatesStrategy;
+import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDParameterUpdate;
+import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDUpdateCalculator;
+import org.apache.ignite.ml.regressions.logistic.binomial.LogisticRegressionModel;
+import org.apache.ignite.ml.regressions.logistic.binomial.LogisticRegressionSGDTrainer;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Tests for {@link OneVsRestTrainer}.
+ */
+public class OneVsRestTrainerTest extends TrainerTest {
+    /**
+     * Test trainer on 2 linearly separable sets.
+     */
+    @Test
+    public void testTrainWithTheLinearlySeparableCase() {
+        Map<Integer, double[]> cacheMock = new HashMap<>();
+
+        for (int i = 0; i < twoLinearlySeparableClasses.length; i++)
+            cacheMock.put(i, twoLinearlySeparableClasses[i]);
+
+        LogisticRegressionSGDTrainer<?> binaryTrainer = new LogisticRegressionSGDTrainer<>()
+            .withUpdatesStgy(new UpdatesStrategy<>(new SimpleGDUpdateCalculator(0.2),
+                SimpleGDParameterUpdate::sumLocal, SimpleGDParameterUpdate::avg))
+            .withMaxIterations(1000)
+            .withLocIterations(10)
+            .withBatchSize(100)
+            .withSeed(123L);
+
+        OneVsRestTrainer<LogisticRegressionModel> trainer = new OneVsRestTrainer<>(binaryTrainer);
+
+        MultiClassModel mdl = trainer.fit(
+            cacheMock,
+            parts,
+            (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)),
+            (k, v) -> v[0]
+        );
+
+        Assert.assertTrue(mdl.toString().length() > 0);
+        Assert.assertTrue(mdl.toString(true).length() > 0);
+        Assert.assertTrue(mdl.toString(false).length() > 0);
+
+        TestUtils.assertEquals(1, mdl.apply(VectorUtils.of(-100, 0)), PRECISION);
+        TestUtils.assertEquals(0, mdl.apply(VectorUtils.of(100, 0)), PRECISION);
+    }
+
+    /** */
+    @Test
+    public void testUpdate() {
+        Map<Integer, double[]> cacheMock = new HashMap<>();
+
+        for (int i = 0; i < twoLinearlySeparableClasses.length; i++)
+            cacheMock.put(i, twoLinearlySeparableClasses[i]);
+
+        LogisticRegressionSGDTrainer<?> binaryTrainer = new LogisticRegressionSGDTrainer<>()
+            .withUpdatesStgy(new UpdatesStrategy<>(new SimpleGDUpdateCalculator(0.2),
+                SimpleGDParameterUpdate::sumLocal, SimpleGDParameterUpdate::avg))
+            .withMaxIterations(1000)
+            .withLocIterations(10)
+            .withBatchSize(100)
+            .withSeed(123L);
+
+        OneVsRestTrainer<LogisticRegressionModel> trainer = new OneVsRestTrainer<>(binaryTrainer);
+
+        MultiClassModel originalMdl = trainer.fit(
+            cacheMock,
+            parts,
+            (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)),
+            (k, v) -> v[0]
+        );
+
+        MultiClassModel updatedOnSameDS = trainer.update(
+            originalMdl,
+            cacheMock,
+            parts,
+            (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)),
+            (k, v) -> v[0]
+        );
+
+        MultiClassModel updatedOnEmptyDS = trainer.update(
+            originalMdl,
+            new HashMap<Integer, double[]>(),
+            parts,
+            (k, v) -> VectorUtils.of(Arrays.copyOfRange(v, 1, v.length)),
+            (k, v) -> v[0]
+        );
+
+        List<Vector> vectors = Arrays.asList(
+            VectorUtils.of(-100, 0),
+            VectorUtils.of(100, 0)
+        );
+
+        for (Vector vec : vectors) {
+            TestUtils.assertEquals(originalMdl.apply(vec), updatedOnSameDS.apply(vec), PRECISION);
+            TestUtils.assertEquals(originalMdl.apply(vec), updatedOnEmptyDS.apply(vec), PRECISION);
+        }
+    }
+}


[27/50] [abbrv] ignite git commit: IGNITE-10393: MVCC: Fixed streamer with persistence on. This closes #5497.

Posted by ag...@apache.org.
IGNITE-10393: MVCC: Fixed streamer with persistence on. This closes #5497.


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

Branch: refs/heads/ignite-9720
Commit: c63a60a39d4131861c98a84440ecb8c67b10ba25
Parents: 25c41fa
Author: Igor Seliverstov <gv...@gmail.com>
Authored: Tue Nov 27 10:31:01 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Nov 27 10:31:01 2018 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheMapEntry.java     |  2 +-
 .../persistence/pagemem/PageMemoryImpl.java     |  2 +-
 ...aStreamProcessorMvccPersistenceSelfTest.java | 28 +++++++++
 .../DataStreamProcessorPersistenceSelfTest.java | 28 +++++++++
 .../DataStreamProcessorSelfTest.java            | 63 +++++++++++++++++++-
 .../testsuites/IgniteBinaryCacheTestSuite.java  |  2 +
 .../testsuites/IgniteCacheMvccTestSuite.java    |  2 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |  2 +
 8 files changed, 126 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c63a60a3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 5f4f974..bbdff35 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -3478,7 +3478,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                             expireTime,
                             partition(),
                             updateCntr,
-                            mvccVer
+                            mvccVer == null ? MvccUtils.INITIAL_VERSION : mvccVer
                         )));
                     } else {
                         cctx.shared().wal().log(new DataRecord(new DataEntry(

http://git-wip-us.apache.org/repos/asf/ignite/blob/c63a60a3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
index f6aa059..c4b0104 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
@@ -53,7 +53,6 @@ import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
-import org.apache.ignite.internal.processors.cache.persistence.StorageException;
 import org.apache.ignite.internal.pagemem.wal.WALIterator;
 import org.apache.ignite.internal.pagemem.wal.WALPointer;
 import org.apache.ignite.internal.pagemem.wal.record.CheckpointRecord;
@@ -65,6 +64,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.persistence.CheckpointLockStateChecker;
 import org.apache.ignite.internal.processors.cache.persistence.CheckpointWriteProgressSupplier;
 import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetricsImpl;
+import org.apache.ignite.internal.processors.cache.persistence.StorageException;
 import org.apache.ignite.internal.processors.cache.persistence.freelist.io.PagesListMetaIO;
 import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO;

http://git-wip-us.apache.org/repos/asf/ignite/blob/c63a60a3/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorMvccPersistenceSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorMvccPersistenceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorMvccPersistenceSelfTest.java
new file mode 100644
index 0000000..9360cab
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorMvccPersistenceSelfTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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.internal.processors.datastreamer;
+
+/**
+ *
+ */
+public class DataStreamProcessorMvccPersistenceSelfTest extends DataStreamProcessorMvccSelfTest {
+    /** {@inheritDoc} */
+    @Override public boolean persistenceEnabled() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c63a60a3/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorPersistenceSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorPersistenceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorPersistenceSelfTest.java
new file mode 100644
index 0000000..7ce4fdd
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorPersistenceSelfTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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.internal.processors.datastreamer;
+
+/**
+ *
+ */
+public class DataStreamProcessorPersistenceSelfTest extends DataStreamProcessorSelfTest {
+    /** {@inheritDoc} */
+    @Override public boolean persistenceEnabled() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c63a60a3/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
index 877df2e..39f43e3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
@@ -42,9 +42,12 @@ import org.apache.ignite.cache.store.CacheStore;
 import org.apache.ignite.cache.store.CacheStoreAdapter;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.IgniteReflectionFactory;
 import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
@@ -52,6 +55,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
+import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteClosure;
@@ -97,6 +101,13 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
     /** */
     private TestStore store;
 
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        if (persistenceEnabled())
+            cleanPersistenceDir();
+    }
+
     /** {@inheritDoc} */
     @Override public void afterTest() throws Exception {
         super.afterTest();
@@ -104,6 +115,13 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
         useCache = false;
     }
 
+    /**
+     * @return {@code True} if persistent store is enabled for test.
+     */
+    public boolean persistenceEnabled() {
+        return false;
+    }
+
     /** {@inheritDoc} */
     @SuppressWarnings({"unchecked"})
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
@@ -141,6 +159,12 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
             }
 
             cfg.setCacheConfiguration(cc);
+
+            if (persistenceEnabled())
+                cfg.setDataStorageConfiguration(new DataStorageConfiguration()
+                    .setDefaultDataRegionConfiguration(new DataRegionConfiguration()
+                            .setPersistenceEnabled(true))
+                    .setWalMode(WALMode.LOG_ONLY));
         }
         else {
             cfg.setCacheConfiguration();
@@ -225,6 +249,8 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
 
             Ignite igniteWithoutCache = startGrid(1);
 
+            afterGridStarted();
+
             final IgniteDataStreamer<Integer, Integer> ldr = igniteWithoutCache.dataStreamer(DEFAULT_CACHE_NAME);
 
             ldr.receiver(DataStreamerCacheUpdaters.<Integer, Integer>batchedSorted());
@@ -337,7 +363,7 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
             startGrid(1);
             startGrid(2);
 
-            awaitPartitionMapExchange();
+            afterGridStarted();
 
             IgniteCache<Integer, Integer> cache = grid(0).cache(DEFAULT_CACHE_NAME);
 
@@ -422,6 +448,8 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
             Ignite g1 = startGrid(1);
             startGrid(2); // Reproduced only for several nodes in topology (if marshalling is used).
 
+            afterGridStarted();
+
             List<Object> arrays = Arrays.<Object>asList(
                 new byte[] {1}, new boolean[] {true, false}, new char[] {2, 3}, new short[] {3, 4},
                 new int[] {4, 5}, new long[] {5, 6}, new float[] {6, 7}, new double[] {7, 8});
@@ -485,6 +513,8 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
 
             Ignite g1 = grid(idx - 1);
 
+            afterGridStarted();
+
             // Get and configure loader.
             final IgniteDataStreamer<Integer, Integer> ldr = g1.dataStreamer(DEFAULT_CACHE_NAME);
 
@@ -589,6 +619,8 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
         try {
             Ignite g1 = startGrid(1);
 
+            afterGridStarted();
+
             IgniteDataStreamer<Object, Object> ldr = g1.dataStreamer(DEFAULT_CACHE_NAME);
 
             ldr.close(false);
@@ -746,6 +778,8 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
         try {
             Ignite g = startGrid();
 
+            afterGridStarted();
+
             final IgniteCache<Integer, Integer> c = g.cache(DEFAULT_CACHE_NAME);
 
             final IgniteDataStreamer<Integer, Integer> ldr = g.dataStreamer(DEFAULT_CACHE_NAME);
@@ -799,6 +833,8 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
         try {
             Ignite g = startGrid();
 
+            afterGridStarted();
+
             IgniteCache<Integer, Integer> c = g.cache(DEFAULT_CACHE_NAME);
 
             IgniteDataStreamer<Integer, Integer> ldr = g.dataStreamer(DEFAULT_CACHE_NAME);
@@ -835,6 +871,8 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
         try {
             Ignite g = startGrid();
 
+            afterGridStarted();
+
             final CountDownLatch latch = new CountDownLatch(9);
 
             g.events().localListen(new IgnitePredicate<Event>() {
@@ -891,6 +929,8 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
             startGrid(2);
             startGrid(3);
 
+            afterGridStarted();
+
             for (int i = 0; i < 1000; i++)
                 storeMap.put(i, i);
 
@@ -940,6 +980,8 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
         }
         finally {
             storeMap = null;
+
+            stopAllGrids();
         }
     }
 
@@ -955,6 +997,8 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
             startGrid(2);
             startGrid(3);
 
+            afterGridStarted();
+
             try (IgniteDataStreamer<String, TestObject> ldr = ignite.dataStreamer(DEFAULT_CACHE_NAME)) {
                 ldr.allowOverwrite(true);
                 ldr.keepBinary(customKeepBinary());
@@ -988,6 +1032,8 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
 
             Ignite ignite = startGrid(1);
 
+            afterGridStarted();
+
             final IgniteCache<String, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
             try (IgniteDataStreamer<String, String> ldr = ignite.dataStreamer(DEFAULT_CACHE_NAME)) {
@@ -1034,6 +1080,8 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
 
             Ignite client = startGrid(0);
 
+            afterGridStarted();
+
             final IgniteCache<String, String> cache = ignite.cache(DEFAULT_CACHE_NAME);
 
             try (IgniteDataStreamer<String, String> ldr = client.dataStreamer(DEFAULT_CACHE_NAME)) {
@@ -1100,6 +1148,19 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Activates grid if necessary and wait for partition map exchange.
+     */
+    private void afterGridStarted() throws InterruptedException {
+        G.allGrids().stream()
+            .filter(g -> !g.cluster().node().isClient())
+            .findAny()
+            .filter(g -> !g.cluster().active())
+            .ifPresent(g -> g.cluster().active(true));
+
+        awaitPartitionMapExchange();
+    }
+
+    /**
      *
      */
     @SuppressWarnings("PublicInnerClass")

http://git-wip-us.apache.org/repos/asf/ignite/blob/c63a60a3/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheTestSuite.java
index 170bb33..5cfe534 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheTestSuite.java
@@ -34,6 +34,7 @@ import org.apache.ignite.internal.processors.cache.binary.distributed.dht.GridCa
 import org.apache.ignite.internal.processors.cache.binary.distributed.dht.GridCacheAtomicPartitionedOnlyBinaryMultithreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.binary.distributed.dht.GridCacheBinariesNearPartitionedByteArrayValuesSelfTest;
 import org.apache.ignite.internal.processors.cache.binary.distributed.dht.GridCacheBinariesPartitionedOnlyByteArrayValuesSelfTest;
+import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessorPersistenceSelfTest;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessorSelfTest;
 
 /**
@@ -51,6 +52,7 @@ public class IgniteBinaryCacheTestSuite extends TestSuite {
 
         // Tests below have a special version for Binary Marshaller
         ignoredTests.add(DataStreamProcessorSelfTest.class);
+        ignoredTests.add(DataStreamProcessorPersistenceSelfTest.class);
         ignoredTests.add(GridCacheAffinityRoutingSelfTest.class);
         ignoredTests.add(IgniteCacheAtomicLocalExpiryPolicyTest.class);
         ignoredTests.add(GridCacheEntryMemorySizeSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c63a60a3/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite.java
index d4b837c..930706d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite.java
@@ -38,6 +38,7 @@ import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccTxFailoverTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccVacuumTest;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccCachePeekTest;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccUnsupportedTxModesTest;
+import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessorMvccPersistenceSelfTest;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessorMvccSelfTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
@@ -59,6 +60,7 @@ public class IgniteCacheMvccTestSuite extends TestSuite {
         suite.addTestSuite(CacheMvccConfigurationValidationTest.class);
 
         suite.addTestSuite(DataStreamProcessorMvccSelfTest.class);
+        suite.addTestSuite(DataStreamProcessorMvccPersistenceSelfTest.class);
         suite.addTestSuite(CacheMvccOperationChecksTest.class);
 
         suite.addTestSuite(CacheMvccRemoteTxOnNearNodeStartTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c63a60a3/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 52e2ba2..dd03ef3 100755
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -144,6 +144,7 @@ import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCa
 import org.apache.ignite.internal.processors.cache.local.GridCacheLocalTxExceptionSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheEntryProcessorExternalizableFailedTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheEntryProcessorNonSerializableTest;
+import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessorPersistenceSelfTest;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessorSelfTest;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerClientReconnectAfterClusterRestartTest;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerImplSelfTest;
@@ -250,6 +251,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheBalancingStoreSelfTest.class);
         suite.addTestSuite(GridCacheAffinityApiSelfTest.class);
         suite.addTestSuite(GridCacheStoreValueBytesSelfTest.class);
+        GridTestUtils.addTestIfNeeded(suite, DataStreamProcessorPersistenceSelfTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, DataStreamProcessorSelfTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, DataStreamerUpdateAfterLoadTest.class, ignoredTests);
         suite.addTestSuite(DataStreamerMultiThreadedSelfTest.class);


[21/50] [abbrv] ignite git commit: IGNITE-10184 enable of test after IGNITE-5380 uncomment after fix. - Fixes #5390.

Posted by ag...@apache.org.
IGNITE-10184 enable of test after IGNITE-5380 uncomment after fix. - Fixes #5390.

Signed-off-by: Dmitriy Pavlov <dp...@apache.org>


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

Branch: refs/heads/ignite-9720
Commit: c870500a4144168e31a75046f848cba741dba96d
Parents: 7e1d178
Author: Albert Iskhakov <al...@gmail.com>
Authored: Mon Nov 26 17:28:38 2018 +0300
Committer: Dmitriy Pavlov <dp...@apache.org>
Committed: Mon Nov 26 17:28:38 2018 +0300

----------------------------------------------------------------------
 .../processors/query/SqlSchemaSelfTest.java     | 25 +++++++++++++-------
 1 file changed, 17 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c870500a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlSchemaSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlSchemaSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlSchemaSelfTest.java
index b271d80..dcfffd8 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlSchemaSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlSchemaSelfTest.java
@@ -20,14 +20,18 @@ package org.apache.ignite.internal.processors.query;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
+import java.util.concurrent.Callable;
 import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.CacheException;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.query.schema.SchemaOperationException;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
@@ -292,19 +296,24 @@ public class SqlSchemaSelfTest extends GridCommonAbstractTest {
      *
      * @throws Exception If failed.
      */
-    public void _testTypeConflictInPublicSchema() throws Exception {
-        // TODO: IGNITE-5380: uncomment work after fix.
-        fail("Hang for now, need to fix");
-
+    public void testTypeConflictInPublicSchema() throws Exception {
         node.createCache(new CacheConfiguration<PersonKey, Person>()
             .setName(CACHE_PERSON)
             .setIndexedTypes(PersonKey.class, Person.class)
             .setSqlSchema(QueryUtils.DFLT_SCHEMA));
 
-        node.createCache(new CacheConfiguration<PersonKey, Person>()
-            .setName(CACHE_PERSON_2)
-            .setIndexedTypes(PersonKey.class, Person.class)
-            .setSqlSchema(QueryUtils.DFLT_SCHEMA));
+        Throwable th = GridTestUtils.assertThrows(log, (Callable<Void>) () -> {
+            node.createCache(new CacheConfiguration<PersonKey, Person>()
+                .setName(CACHE_PERSON_2)
+                .setIndexedTypes(PersonKey.class, Person.class)
+                .setSqlSchema(QueryUtils.DFLT_SCHEMA));
+
+            return null;
+        }, CacheException.class, null);
+
+        SchemaOperationException e = X.cause(th, SchemaOperationException.class);
+
+        assertEquals(SchemaOperationException.CODE_TABLE_EXISTS, e.code());
     }
 
     /**


[03/50] [abbrv] ignite git commit: IGNITE-10335: move ML examples datasets files to resources

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/resources/datasets/titanic.csv
----------------------------------------------------------------------
diff --git a/examples/src/main/resources/datasets/titanic.csv b/examples/src/main/resources/datasets/titanic.csv
new file mode 100644
index 0000000..6994016
--- /dev/null
+++ b/examples/src/main/resources/datasets/titanic.csv
@@ -0,0 +1,1310 @@
+pclass;survived;name;sex;age;sibsp;parch;ticket;fare;cabin;embarked;boat;body;homedest
+1;1;Allen, Miss. Elisabeth Walton;;29;;;24160;211,3375;B5;;2;;St Louis, MO
+1;1;Allison, Master. Hudson Trevor;male;0,9167;1;2;113781;151,55;C22 C26;AA;11;;Montreal, PQ / Chesterville, ON
+1;0;Allison, Miss. Helen Loraine;female;2;1;2;113781;151,55;C22 C26;S;;;Montreal, PQ / Chesterville, ON
+1;0;Allison, Mr. Hudson Joshua Creighton;male;30;1;2;113781;151,55;C22 C26;S;;135;Montreal, PQ / Chesterville, ON
+1;0;Allison, Mrs. Hudson J C (Bessie Waldo Daniels);female;25;1;2;113781;151,55;C22 C26;S;;;Montreal, PQ / Chesterville, ON
+1;1;Anderson, Mr. Harry;male;48;0;0;19952;26,55;E12;S;3;;New York, NY
+1;1;Andrews, Miss. Kornelia Theodosia;female;63;1;0;13502;77,9583;D7;S;10;;Hudson, NY
+1;0;Andrews, Mr. Thomas Jr;male;39;0;0;112050;0;A36;S;;;Belfast, NI
+1;1;Appleton, Mrs. Edward Dale (Charlotte Lamson);female;53;2;0;11769;51,4792;C101;S;D;;Bayside, Queens, NY
+1;0;Artagaveytia, Mr. Ramon;male;71;0;0;PC 17609;49,5042;;C;;22;Montevideo, Uruguay
+1;0;Astor, Col. John Jacob;male;47;1;0;PC 17757;227,525;C62 C64;C;;124;New York, NY
+1;1;Astor, Mrs. John Jacob (Madeleine Talmadge Force);female;18;1;0;PC 17757;227,525;C62 C64;C;4;;New York, NY
+1;1;Aubart, Mme. Leontine Pauline;female;24;0;0;PC 17477;69,3;B35;C;9;;Paris, France
+1;1;"Barber, Miss. Ellen ""Nellie""";female;26;0;0;19877;78,85;;S;6;;
+1;1;Barkworth, Mr. Algernon Henry Wilson;male;80;0;0;27042;30;A23;S;B;;Hessle, Yorks
+1;0;Baumann, Mr. John D;male;;0;0;PC 17318;25,925;;S;;;New York, NY
+1;0;Baxter, Mr. Quigg Edmond;male;24;0;1;PC 17558;247,5208;B58 B60;C;;;Montreal, PQ
+1;1;Baxter, Mrs. James (Helene DeLaudeniere Chaput);female;50;0;1;PC 17558;247,5208;B58 B60;C;6;;Montreal, PQ
+1;1;Bazzani, Miss. Albina;female;32;0;0;11813;76,2917;D15;C;8;;
+1;0;Beattie, Mr. Thomson;male;36;0;0;13050;75,2417;C6;C;A;;Winnipeg, MN
+1;1;Beckwith, Mr. Richard Leonard;male;37;1;1;11751;52,5542;D35;S;5;;New York, NY
+1;1;Beckwith, Mrs. Richard Leonard (Sallie Monypeny);female;47;1;1;11751;52,5542;D35;S;5;;New York, NY
+1;1;Behr, Mr. Karl Howell;male;26;0;0;111369;30;C148;C;5;;New York, NY
+1;1;Bidois, Miss. Rosalie;female;42;0;0;PC 17757;227,525;;C;4;;
+1;1;Bird, Miss. Ellen;female;29;0;0;PC 17483;221,7792;C97;S;8;;
+1;0;Birnbaum, Mr. Jakob;male;25;0;0;13905;26;;C;;148;San Francisco, CA
+1;1;Bishop, Mr. Dickinson H;male;25;1;0;11967;91,0792;B49;C;7;;Dowagiac, MI
+1;1;Bishop, Mrs. Dickinson H (Helen Walton);female;19;1;0;11967;91,0792;B49;C;7;;Dowagiac, MI
+1;1;Bissette, Miss. Amelia;female;35;0;0;PC 17760;135,6333;C99;S;8;;
+1;1;Bjornstrom-Steffansson, Mr. Mauritz Hakan;male;28;0;0;110564;26,55;C52;S;D;;Stockholm, Sweden / Washington, DC
+1;0;Blackwell, Mr. Stephen Weart;male;45;0;0;113784;35,5;T;S;;;Trenton, NJ
+1;1;Blank, Mr. Henry;male;40;0;0;112277;31;A31;C;7;;Glen Ridge, NJ
+1;1;Bonnell, Miss. Caroline;female;30;0;0;36928;164,8667;C7;S;8;;Youngstown, OH
+1;1;Bonnell, Miss. Elizabeth;female;58;0;0;113783;26,55;C103;S;8;;Birkdale, England Cleveland, Ohio
+1;0;Borebank, Mr. John James;male;42;0;0;110489;26,55;D22;S;;;London / Winnipeg, MB
+1;1;Bowen, Miss. Grace Scott;female;45;0;0;PC 17608;262,375;;C;4;;Cooperstown, NY
+1;1;Bowerman, Miss. Elsie Edith;female;22;0;1;113505;55;E33;S;6;;St Leonards-on-Sea, England Ohio
+1;1;"Bradley, Mr. George (""George Arthur Brayton"")";male;;0;0;111427;26,55;;S;9;;Los Angeles, CA
+1;0;Brady, Mr. John Bertram;male;41;0;0;113054;30,5;A21;S;;;Pomeroy, WA
+1;0;Brandeis, Mr. Emil;male;48;0;0;PC 17591;50,4958;B10;C;;208;Omaha, NE
+1;0;Brewe, Dr. Arthur Jackson;male;;0;0;112379;39,6;;C;;;Philadelphia, PA
+1;1;Brown, Mrs. James Joseph (Margaret Tobin);female;44;0;0;PC 17610;27,7208;B4;C;6;;Denver, CO
+1;1;Brown, Mrs. John Murray (Caroline Lane Lamson);female;59;2;0;11769;51,4792;C101;S;D;;Belmont, MA
+1;1;Bucknell, Mrs. William Robert (Emma Eliza Ward);female;60;0;0;11813;76,2917;D15;C;8;;Philadelphia, PA
+1;1;Burns, Miss. Elizabeth Margaret;female;41;0;0;16966;134,5;E40;C;3;;
+1;0;Butt, Major. Archibald Willingham;male;45;0;0;113050;26,55;B38;S;;;Washington, DC
+1;0;Cairns, Mr. Alexander;male;;0;0;113798;31;;S;;;
+1;1;Calderhead, Mr. Edward Pennington;male;42;0;0;PC 17476;26,2875;E24;S;5;;New York, NY
+1;1;Candee, Mrs. Edward (Helen Churchill Hungerford);female;53;0;0;PC 17606;27,4458;;C;6;;Washington, DC
+1;1;Cardeza, Mr. Thomas Drake Martinez;male;36;0;1;PC 17755;512,3292;B51 B53 B55;C;3;;Austria-Hungary / Germantown, Philadelphia, PA
+1;1;Cardeza, Mrs. James Warburton Martinez (Charlotte Wardle Drake);female;58;0;1;PC 17755;512,3292;B51 B53 B55;C;3;;Germantown, Philadelphia, PA
+1;0;Carlsson, Mr. Frans Olof;male;33;0;0;695;5;B51 B53 B55;S;;;New York, NY
+1;0;Carrau, Mr. Francisco M;male;28;0;0;113059;47,1;;S;;;Montevideo, Uruguay
+1;0;Carrau, Mr. Jose Pedro;male;17;0;0;113059;47,1;;S;;;Montevideo, Uruguay
+1;1;Carter, Master. William Thornton II;male;11;1;2;113760;120;B96 B98;S;4;;Bryn Mawr, PA
+1;1;Carter, Miss. Lucile Polk;female;14;1;2;113760;120;B96 B98;S;4;;Bryn Mawr, PA
+1;1;Carter, Mr. William Ernest;male;36;1;2;113760;120;B96 B98;S;C;;Bryn Mawr, PA
+1;1;Carter, Mrs. William Ernest (Lucile Polk);female;36;1;2;113760;120;B96 B98;S;4;;Bryn Mawr, PA
+1;0;Case, Mr. Howard Brown;male;49;0;0;19924;26;;S;;;Ascot, Berkshire / Rochester, NY
+1;1;Cassebeer, Mrs. Henry Arthur Jr (Eleanor Genevieve Fosdick);female;;0;0;17770;27,7208;;C;5;;New York, NY
+1;0;Cavendish, Mr. Tyrell William;male;36;1;0;19877;78,85;C46;S;;172;Little Onn Hall, Staffs
+1;1;Cavendish, Mrs. Tyrell William (Julia Florence Siegel);female;76;1;0;19877;78,85;C46;S;6;;Little Onn Hall, Staffs
+1;0;Chaffee, Mr. Herbert Fuller;male;46;1;0;W.E.P. 5734;61,175;E31;S;;;Amenia, ND
+1;1;Chaffee, Mrs. Herbert Fuller (Carrie Constance Toogood);female;47;1;0;W.E.P. 5734;61,175;E31;S;4;;Amenia, ND
+1;1;Chambers, Mr. Norman Campbell;male;27;1;0;113806;53,1;E8;S;5;;New York, NY / Ithaca, NY
+1;1;Chambers, Mrs. Norman Campbell (Bertha Griggs);female;33;1;0;113806;53,1;E8;S;5;;New York, NY / Ithaca, NY
+1;1;Chaudanson, Miss. Victorine;female;36;0;0;PC 17608;262,375;B61;C;4;;
+1;1;Cherry, Miss. Gladys;female;30;0;0;110152;86,5;B77;S;8;;London, England
+1;1;Chevre, Mr. Paul Romaine;male;45;0;0;PC 17594;29,7;A9;C;7;;Paris, France
+1;1;Chibnall, Mrs. (Edith Martha Bowerman);female;;0;1;113505;55;E33;S;6;;St Leonards-on-Sea, England Ohio
+1;0;Chisholm, Mr. Roderick Robert Crispin;male;;0;0;112051;0;;S;;;Liverpool, England / Belfast
+1;0;Clark, Mr. Walter Miller;male;27;1;0;13508;136,7792;C89;C;;;Los Angeles, CA
+1;1;Clark, Mrs. Walter Miller (Virginia McDowell);female;26;1;0;13508;136,7792;C89;C;4;;Los Angeles, CA
+1;1;Cleaver, Miss. Alice;female;22;0;0;113781;151,55;;S;11;;
+1;0;Clifford, Mr. George Quincy;male;;0;0;110465;52;A14;S;;;Stoughton, MA
+1;0;Colley, Mr. Edward Pomeroy;male;47;0;0;5727;25,5875;E58;S;;;Victoria, BC
+1;1;Compton, Miss. Sara Rebecca;female;39;1;1;PC 17756;83,1583;E49;C;14;;Lakewood, NJ
+1;0;Compton, Mr. Alexander Taylor Jr;male;37;1;1;PC 17756;83,1583;E52;C;;;Lakewood, NJ
+1;1;Compton, Mrs. Alexander Taylor (Mary Eliza Ingersoll);female;64;0;2;PC 17756;83,1583;E45;C;14;;Lakewood, NJ
+1;1;Cornell, Mrs. Robert Clifford (Malvina Helen Lamson);female;55;2;0;11770;25,7;C101;S;2;;New York, NY
+1;0;Crafton, Mr. John Bertram;male;;0;0;113791;26,55;;S;;;Roachdale, IN
+1;0;Crosby, Capt. Edward Gifford;male;70;1;1;WE/P 5735;71;B22;S;;269;Milwaukee, WI
+1;1;Crosby, Miss. Harriet R;female;36;0;2;WE/P 5735;71;B22;S;7;;Milwaukee, WI
+1;1;Crosby, Mrs. Edward Gifford (Catherine Elizabeth Halstead);female;64;1;1;112901;26,55;B26;S;7;;Milwaukee, WI
+1;0;Cumings, Mr. John Bradley;male;39;1;0;PC 17599;71,2833;C85;C;;;New York, NY
+1;1;Cumings, Mrs. John Bradley (Florence Briggs Thayer);female;38;1;0;PC 17599;71,2833;C85;C;4;;New York, NY
+1;1;Daly, Mr. Peter Denis ;male;51;0;0;113055;26,55;E17;S;5 9;;Lima, Peru
+1;1;Daniel, Mr. Robert Williams;male;27;0;0;113804;30,5;;S;3;;Philadelphia, PA
+1;1;Daniels, Miss. Sarah;female;33;0;0;113781;151,55;;S;8;;
+1;0;Davidson, Mr. Thornton;male;31;1;0;F.C. 12750;52;B71;S;;;Montreal, PQ
+1;1;Davidson, Mrs. Thornton (Orian Hays);female;27;1;2;F.C. 12750;52;B71;S;3;;Montreal, PQ
+1;1;Dick, Mr. Albert Adrian;male;31;1;0;17474;57;B20;S;3;;Calgary, AB
+1;1;Dick, Mrs. Albert Adrian (Vera Gillespie);female;17;1;0;17474;57;B20;S;3;;Calgary, AB
+1;1;Dodge, Dr. Washington;male;53;1;1;33638;81,8583;A34;S;13;;San Francisco, CA
+1;1;Dodge, Master. Washington;male;4;0;2;33638;81,8583;A34;S;5;;San Francisco, CA
+1;1;Dodge, Mrs. Washington (Ruth Vidaver);female;54;1;1;33638;81,8583;A34;S;5;;San Francisco, CA
+1;0;Douglas, Mr. Walter Donald;male;50;1;0;PC 17761;106,425;C86;C;;62;Deephaven, MN / Cedar Rapids, IA
+1;1;Douglas, Mrs. Frederick Charles (Mary Helene Baxter);female;27;1;1;PC 17558;247,5208;B58 B60;C;6;;Montreal, PQ
+1;1;Douglas, Mrs. Walter Donald (Mahala Dutton);female;48;1;0;PC 17761;106,425;C86;C;2;;Deephaven, MN / Cedar Rapids, IA
+1;1;"Duff Gordon, Lady. (Lucille Christiana Sutherland) (""Mrs Morgan"")";female;48;1;0;11755;39,6;A16;C;1;;London / Paris
+1;1;"Duff Gordon, Sir. Cosmo Edmund (""Mr Morgan"")";male;49;1;0;PC 17485;56,9292;A20;C;1;;London / Paris
+1;0;Dulles, Mr. William Crothers;male;39;0;0;PC 17580;29,7;A18;C;;133;Philadelphia, PA
+1;1;Earnshaw, Mrs. Boulton (Olive Potter);female;23;0;1;11767;83,1583;C54;C;7;;Mt Airy, Philadelphia, PA
+1;1;Endres, Miss. Caroline Louise;female;38;0;0;PC 17757;227,525;C45;C;4;;New York, NY
+1;1;Eustis, Miss. Elizabeth Mussey;female;54;1;0;36947;78,2667;D20;C;4;;Brookline, MA
+1;0;Evans, Miss. Edith Corse;female;36;0;0;PC 17531;31,6792;A29;C;;;New York, NY
+1;0;Farthing, Mr. John;male;;0;0;PC 17483;221,7792;C95;S;;;
+1;1;Flegenheim, Mrs. Alfred (Antoinette);female;;0;0;PC 17598;31,6833;;S;7;;New York, NY
+1;1;Fleming, Miss. Margaret;female;;0;0;17421;110,8833;;C;4;;
+1;1;"Flynn, Mr. John Irwin (""Irving"")";male;36;0;0;PC 17474;26,3875;E25;S;5;;Brooklyn, NY
+1;0;Foreman, Mr. Benjamin Laventall;male;30;0;0;113051;27,75;C111;C;;;New York, NY
+1;1;Fortune, Miss. Alice Elizabeth;female;24;3;2;19950;263;C23 C25 C27;S;10;;Winnipeg, MB
+1;1;Fortune, Miss. Ethel Flora;female;28;3;2;19950;263;C23 C25 C27;S;10;;Winnipeg, MB
+1;1;Fortune, Miss. Mabel Helen;female;23;3;2;19950;263;C23 C25 C27;S;10;;Winnipeg, MB
+1;0;Fortune, Mr. Charles Alexander;male;19;3;2;19950;263;C23 C25 C27;S;;;Winnipeg, MB
+1;0;Fortune, Mr. Mark;male;64;1;4;19950;263;C23 C25 C27;S;;;Winnipeg, MB
+1;1;Fortune, Mrs. Mark (Mary McDougald);female;60;1;4;19950;263;C23 C25 C27;S;10;;Winnipeg, MB
+1;1;Francatelli, Miss. Laura Mabel;female;30;0;0;PC 17485;56,9292;E36;C;1;;
+1;0;Franklin, Mr. Thomas Parham;male;;0;0;113778;26,55;D34;S;;;Westcliff-on-Sea, Essex
+1;1;Frauenthal, Dr. Henry William;male;50;2;0;PC 17611;133,65;;S;5;;New York, NY
+1;1;Frauenthal, Mr. Isaac Gerald;male;43;1;0;17765;27,7208;D40;C;5;;New York, NY
+1;1;Frauenthal, Mrs. Henry William (Clara Heinsheimer);female;;1;0;PC 17611;133,65;;S;5;;New York, NY
+1;1;Frolicher, Miss. Hedwig Margaritha;female;22;0;2;13568;49,5;B39;C;5;;Zurich, Switzerland
+1;1;Frolicher-Stehli, Mr. Maxmillian;male;60;1;1;13567;79,2;B41;C;5;;Zurich, Switzerland
+1;1;Frolicher-Stehli, Mrs. Maxmillian (Margaretha Emerentia Stehli);female;48;1;1;13567;79,2;B41;C;5;;Zurich, Switzerland
+1;0;Fry, Mr. Richard;male;;0;0;112058;0;B102;S;;;
+1;0;Futrelle, Mr. Jacques Heath;male;37;1;0;113803;53,1;C123;S;;;Scituate, MA
+1;1;Futrelle, Mrs. Jacques Heath (Lily May Peel);female;35;1;0;113803;53,1;C123;S;D;;Scituate, MA
+1;0;Gee, Mr. Arthur H;male;47;0;0;111320;38,5;E63;S;;275;St Anne's-on-Sea, Lancashire
+1;1;Geiger, Miss. Amalie;female;35;0;0;113503;211,5;C130;C;4;;
+1;1;Gibson, Miss. Dorothy Winifred;female;22;0;1;112378;59,4;;C;7;;New York, NY
+1;1;Gibson, Mrs. Leonard (Pauline C Boeson);female;45;0;1;112378;59,4;;C;7;;New York, NY
+1;0;Giglio, Mr. Victor;male;24;0;0;PC 17593;79,2;B86;C;;;
+1;1;Goldenberg, Mr. Samuel L;male;49;1;0;17453;89,1042;C92;C;5;;Paris, France / New York, NY
+1;1;Goldenberg, Mrs. Samuel L (Edwiga Grabowska);female;;1;0;17453;89,1042;C92;C;5;;Paris, France / New York, NY
+1;0;Goldschmidt, Mr. George B;male;71;0;0;PC 17754;34,6542;A5;C;;;New York, NY
+1;1;Gracie, Col. Archibald IV;male;53;0;0;113780;28,5;C51;C;B;;Washington, DC
+1;1;Graham, Miss. Margaret Edith;female;19;0;0;112053;30;B42;S;3;;Greenwich, CT
+1;0;Graham, Mr. George Edward;male;38;0;1;PC 17582;153,4625;C91;S;;147;Winnipeg, MB
+1;1;Graham, Mrs. William Thompson (Edith Junkins);female;58;0;1;PC 17582;153,4625;C125;S;3;;Greenwich, CT
+1;1;Greenfield, Mr. William Bertram;male;23;0;1;PC 17759;63,3583;D10 D12;C;7;;New York, NY
+1;1;Greenfield, Mrs. Leo David (Blanche Strouse);female;45;0;1;PC 17759;63,3583;D10 D12;C;7;;New York, NY
+1;0;Guggenheim, Mr. Benjamin;male;46;0;0;PC 17593;79,2;B82 B84;C;;;New York, NY
+1;1;Harder, Mr. George Achilles;male;25;1;0;11765;55,4417;E50;C;5;;Brooklyn, NY
+1;1;Harder, Mrs. George Achilles (Dorothy Annan);female;25;1;0;11765;55,4417;E50;C;5;;Brooklyn, NY
+1;1;Harper, Mr. Henry Sleeper;male;48;1;0;PC 17572;76,7292;D33;C;3;;New York, NY
+1;1;Harper, Mrs. Henry Sleeper (Myna Haxtun);female;49;1;0;PC 17572;76,7292;D33;C;3;;New York, NY
+1;0;Harrington, Mr. Charles H;male;;0;0;113796;42,4;;S;;;
+1;0;Harris, Mr. Henry Birkhardt;male;45;1;0;36973;83,475;C83;S;;;New York, NY
+1;1;Harris, Mrs. Henry Birkhardt (Irene Wallach);female;35;1;0;36973;83,475;C83;S;D;;New York, NY
+1;0;Harrison, Mr. William;male;40;0;0;112059;0;B94;S;;110;
+1;1;Hassab, Mr. Hammad;male;27;0;0;PC 17572;76,7292;D49;C;3;;
+1;1;Hawksford, Mr. Walter James;male;;0;0;16988;30;D45;S;3;;Kingston, Surrey
+1;1;Hays, Miss. Margaret Bechstein;female;24;0;0;11767;83,1583;C54;C;7;;New York, NY
+1;0;Hays, Mr. Charles Melville;male;55;1;1;12749;93,5;B69;S;;307;Montreal, PQ
+1;1;Hays, Mrs. Charles Melville (Clara Jennings Gregg);female;52;1;1;12749;93,5;B69;S;3;;Montreal, PQ
+1;0;Head, Mr. Christopher;male;42;0;0;113038;42,5;B11;S;;;London / Middlesex
+1;0;Hilliard, Mr. Herbert Henry;male;;0;0;17463;51,8625;E46;S;;;Brighton, MA
+1;0;Hipkins, Mr. William Edward;male;55;0;0;680;50;C39;S;;;London / Birmingham
+1;1;Hippach, Miss. Jean Gertrude;female;16;0;1;111361;57,9792;B18;C;4;;Chicago, IL
+1;1;Hippach, Mrs. Louis Albert (Ida Sophia Fischer);female;44;0;1;111361;57,9792;B18;C;4;;Chicago, IL
+1;1;Hogeboom, Mrs. John C (Anna Andrews);female;51;1;0;13502;77,9583;D11;S;10;;Hudson, NY
+1;0;Holverson, Mr. Alexander Oskar;male;42;1;0;113789;52;;S;;38;New York, NY
+1;1;Holverson, Mrs. Alexander Oskar (Mary Aline Towner);female;35;1;0;113789;52;;S;8;;New York, NY
+1;1;"Homer, Mr. Harry (""Mr E Haven"")";male;35;0;0;111426;26,55;;C;15;;Indianapolis, IN
+1;1;Hoyt, Mr. Frederick Maxfield;male;38;1;0;19943;90;C93;S;D;;New York, NY /  Stamford CT
+1;0;Hoyt, Mr. William Fisher;male;;0;0;PC 17600;30,6958;;C;14;;New York, NY
+1;1;Hoyt, Mrs. Frederick Maxfield (Jane Anne Forby);female;35;1;0;19943;90;C93;S;D;;New York, NY /  Stamford CT
+1;1;Icard, Miss. Amelie;female;38;0;0;113572;80;B28;;6;;
+1;0;Isham, Miss. Ann Elizabeth;female;50;0;0;PC 17595;28,7125;C49;C;;;Paris, France New York, NY
+1;1;Ismay, Mr. Joseph Bruce;male;49;0;0;112058;0;B52 B54 B56;S;C;;Liverpool
+1;0;Jones, Mr. Charles Cresson;male;46;0;0;694;26;;S;;80;Bennington, VT
+1;0;Julian, Mr. Henry Forbes;male;50;0;0;113044;26;E60;S;;;London
+1;0;Keeping, Mr. Edwin;male;32,5;0;0;113503;211,5;C132;C;;45;
+1;0;Kent, Mr. Edward Austin;male;58;0;0;11771;29,7;B37;C;;258;Buffalo, NY
+1;0;Kenyon, Mr. Frederick R;male;41;1;0;17464;51,8625;D21;S;;;Southington / Noank, CT
+1;1;Kenyon, Mrs. Frederick R (Marion);female;;1;0;17464;51,8625;D21;S;8;;Southington / Noank, CT
+1;1;Kimball, Mr. Edwin Nelson Jr;male;42;1;0;11753;52,5542;D19;S;5;;Boston, MA
+1;1;Kimball, Mrs. Edwin Nelson Jr (Gertrude Parsons);female;45;1;0;11753;52,5542;D19;S;5;;Boston, MA
+1;0;Klaber, Mr. Herman;male;;0;0;113028;26,55;C124;S;;;Portland, OR
+1;1;Kreuchen, Miss. Emilie;female;39;0;0;24160;211,3375;;S;2;;
+1;1;Leader, Dr. Alice (Farnham);female;49;0;0;17465;25,9292;D17;S;8;;New York, NY
+1;1;LeRoy, Miss. Bertha;female;30;0;0;PC 17761;106,425;;C;2;;
+1;1;Lesurer, Mr. Gustave J;male;35;0;0;PC 17755;512,3292;B101;C;3;;
+1;0;Lewy, Mr. Ervin G;male;;0;0;PC 17612;27,7208;;C;;;Chicago, IL
+1;0;"Lindeberg-Lind, Mr. Erik Gustaf (""Mr Edward Lingrey"")";male;42;0;0;17475;26,55;;S;;;Stockholm, Sweden
+1;1;Lindstrom, Mrs. Carl Johan (Sigrid Posse);female;55;0;0;112377;27,7208;;C;6;;Stockholm, Sweden
+1;1;Lines, Miss. Mary Conover;female;16;0;1;PC 17592;39,4;D28;S;9;;Paris, France
+1;1;Lines, Mrs. Ernest H (Elizabeth Lindsey James);female;51;0;1;PC 17592;39,4;D28;S;9;;Paris, France
+1;0;Long, Mr. Milton Clyde;male;29;0;0;113501;30;D6;S;;126;Springfield, MA
+1;1;Longley, Miss. Gretchen Fiske;female;21;0;0;13502;77,9583;D9;S;10;;Hudson, NY
+1;0;Loring, Mr. Joseph Holland;male;30;0;0;113801;45,5;;S;;;London / New York, NY
+1;1;Lurette, Miss. Elise;female;58;0;0;PC 17569;146,5208;B80;C;;;
+1;1;Madill, Miss. Georgette Alexandra;female;15;0;1;24160;211,3375;B5;S;2;;St Louis, MO
+1;0;Maguire, Mr. John Edward;male;30;0;0;110469;26;C106;S;;;Brockton, MA
+1;1;Maioni, Miss. Roberta;female;16;0;0;110152;86,5;B79;S;8;;
+1;1;Marechal, Mr. Pierre;male;;0;0;11774;29,7;C47;C;7;;Paris, France
+1;0;Marvin, Mr. Daniel Warner;male;19;1;0;113773;53,1;D30;S;;;New York, NY
+1;1;Marvin, Mrs. Daniel Warner (Mary Graham Carmichael Farquarson);female;18;1;0;113773;53,1;D30;S;10;;New York, NY
+1;1;"Mayne, Mlle. Berthe Antonine (""Mrs de Villiers"")";female;24;0;0;PC 17482;49,5042;C90;C;6;;Belgium  Montreal, PQ
+1;0;McCaffry, Mr. Thomas Francis;male;46;0;0;13050;75,2417;C6;C;;292;Vancouver, BC
+1;0;McCarthy, Mr. Timothy J;male;54;0;0;17463;51,8625;E46;S;;175;Dorchester, MA
+1;1;McGough, Mr. James Robert;male;36;0;0;PC 17473;26,2875;E25;S;7;;Philadelphia, PA
+1;0;Meyer, Mr. Edgar Joseph;male;28;1;0;PC 17604;82,1708;;C;;;New York, NY
+1;1;Meyer, Mrs. Edgar Joseph (Leila Saks);female;;1;0;PC 17604;82,1708;;C;6;;New York, NY
+1;0;Millet, Mr. Francis Davis;male;65;0;0;13509;26,55;E38;S;;249;East Bridgewater, MA
+1;0;Minahan, Dr. William Edward;male;44;2;0;19928;90;C78;Q;;230;Fond du Lac, WI
+1;1;Minahan, Miss. Daisy E;female;33;1;0;19928;90;C78;Q;14;;Green Bay, WI
+1;1;Minahan, Mrs. William Edward (Lillian E Thorpe);female;37;1;0;19928;90;C78;Q;14;;Fond du Lac, WI
+1;1;Mock, Mr. Philipp Edmund;male;30;1;0;13236;57,75;C78;C;11;;New York, NY
+1;0;Molson, Mr. Harry Markland;male;55;0;0;113787;30,5;C30;S;;;Montreal, PQ
+1;0;Moore, Mr. Clarence Bloomfield;male;47;0;0;113796;42,4;;S;;;Washington, DC
+1;0;Natsch, Mr. Charles H;male;37;0;1;PC 17596;29,7;C118;C;;;Brooklyn, NY
+1;1;Newell, Miss. Madeleine;female;31;1;0;35273;113,275;D36;C;6;;Lexington, MA
+1;1;Newell, Miss. Marjorie;female;23;1;0;35273;113,275;D36;C;6;;Lexington, MA
+1;0;Newell, Mr. Arthur Webster;male;58;0;2;35273;113,275;D48;C;;122;Lexington, MA
+1;1;Newsom, Miss. Helen Monypeny;female;19;0;2;11752;26,2833;D47;S;5;;New York, NY
+1;0;Nicholson, Mr. Arthur Ernest;male;64;0;0;693;26;;S;;263;Isle of Wight, England
+1;1;Oliva y Ocana, Dona. Fermina;female;39;0;0;PC 17758;108,9;C105;C;8;;
+1;1;Omont, Mr. Alfred Fernand;male;;0;0;F.C. 12998;25,7417;;C;7;;Paris, France
+1;1;Ostby, Miss. Helene Ragnhild;female;22;0;1;113509;61,9792;B36;C;5;;Providence, RI
+1;0;Ostby, Mr. Engelhart Cornelius;male;65;0;1;113509;61,9792;B30;C;;234;Providence, RI
+1;0;Ovies y Rodriguez, Mr. Servando;male;28,5;0;0;PC 17562;27,7208;D43;C;;189;?Havana, Cuba
+1;0;Parr, Mr. William Henry Marsh;male;;0;0;112052;0;;S;;;Belfast
+1;0;Partner, Mr. Austen;male;45,5;0;0;113043;28,5;C124;S;;166;Surbiton Hill, Surrey
+1;0;Payne, Mr. Vivian Ponsonby;male;23;0;0;12749;93,5;B24;S;;;Montreal, PQ
+1;0;Pears, Mr. Thomas Clinton;male;29;1;0;113776;66,6;C2;S;;;Isleworth, England
+1;1;Pears, Mrs. Thomas (Edith Wearne);female;22;1;0;113776;66,6;C2;S;8;;Isleworth, England
+1;0;Penasco y Castellana, Mr. Victor de Satode;male;18;1;0;PC 17758;108,9;C65;C;;;Madrid, Spain
+1;1;Penasco y Castellana, Mrs. Victor de Satode (Maria Josefa Perez de Soto y Vallejo);female;17;1;0;PC 17758;108,9;C65;C;8;;Madrid, Spain
+1;1;Perreault, Miss. Anne;female;30;0;0;12749;93,5;B73;S;3;;
+1;1;Peuchen, Major. Arthur Godfrey;male;52;0;0;113786;30,5;C104;S;6;;Toronto, ON
+1;0;Porter, Mr. Walter Chamberlain;male;47;0;0;110465;52;C110;S;;207;Worcester, MA
+1;1;Potter, Mrs. Thomas Jr (Lily Alexenia Wilson);female;56;0;1;11767;83,1583;C50;C;7;;Mt Airy, Philadelphia, PA
+1;0;Reuchlin, Jonkheer. John George;male;38;0;0;19972;0;;S;;;Rotterdam, Netherlands
+1;1;Rheims, Mr. George Alexander Lucien;male;;0;0;PC 17607;39,6;;S;A;;Paris /  New York, NY
+1;0;Ringhini, Mr. Sante;male;22;0;0;PC 17760;135,6333;;C;;232;
+1;0;Robbins, Mr. Victor;male;;0;0;PC 17757;227,525;;C;;;
+1;1;Robert, Mrs. Edward Scott (Elisabeth Walton McMillan);female;43;0;1;24160;211,3375;B3;S;2;;St Louis, MO
+1;0;Roebling, Mr. Washington Augustus II;male;31;0;0;PC 17590;50,4958;A24;S;;;Trenton, NJ
+1;1;"Romaine, Mr. Charles Hallace (""Mr C Rolmane"")";male;45;0;0;111428;26,55;;S;9;;New York, NY
+1;0;Rood, Mr. Hugh Roscoe;male;;0;0;113767;50;A32;S;;;Seattle, WA
+1;1;Rosenbaum, Miss. Edith Louise;female;33;0;0;PC 17613;27,7208;A11;C;11;;Paris, France
+1;0;"Rosenshine, Mr. George (""Mr George Thorne"")";male;46;0;0;PC 17585;79,2;;C;;16;New York, NY
+1;0;Ross, Mr. John Hugo;male;36;0;0;13049;40,125;A10;C;;;Winnipeg, MB
+1;1;Rothes, the Countess. of (Lucy Noel Martha Dyer-Edwards);female;33;0;0;110152;86,5;B77;S;8;;London  Vancouver, BC
+1;0;Rothschild, Mr. Martin;male;55;1;0;PC 17603;59,4;;C;;;New York, NY
+1;1;Rothschild, Mrs. Martin (Elizabeth L. Barrett);female;54;1;0;PC 17603;59,4;;C;6;;New York, NY
+1;0;Rowe, Mr. Alfred G;male;33;0;0;113790;26,55;;S;;109;London
+1;1;Ryerson, Master. John Borie;male;13;2;2;PC 17608;262,375;B57 B59 B63 B66;C;4;;Haverford, PA / Cooperstown, NY
+1;1;Ryerson, Miss. Emily Borie;female;18;2;2;PC 17608;262,375;B57 B59 B63 B66;C;4;;Haverford, PA / Cooperstown, NY
+1;1;"Ryerson, Miss. Susan Parker ""Suzette""";female;21;2;2;PC 17608;262,375;B57 B59 B63 B66;C;4;;Haverford, PA / Cooperstown, NY
+1;0;Ryerson, Mr. Arthur Larned;male;61;1;3;PC 17608;262,375;B57 B59 B63 B66;C;;;Haverford, PA / Cooperstown, NY
+1;1;Ryerson, Mrs. Arthur Larned (Emily Maria Borie);female;48;1;3;PC 17608;262,375;B57 B59 B63 B66;C;4;;Haverford, PA / Cooperstown, NY
+1;1;Saalfeld, Mr. Adolphe;male;;0;0;19988;30,5;C106;S;3;;Manchester, England
+1;1;Sagesser, Mlle. Emma;female;24;0;0;PC 17477;69,3;B35;C;9;;
+1;1;Salomon, Mr. Abraham L;male;;0;0;111163;26;;S;1;;New York, NY
+1;1;Schabert, Mrs. Paul (Emma Mock);female;35;1;0;13236;57,75;C28;C;11;;New York, NY
+1;1;Serepeca, Miss. Augusta;female;30;0;0;113798;31;;C;4;;
+1;1;Seward, Mr. Frederic Kimber;male;34;0;0;113794;26,55;;S;7;;New York, NY
+1;1;Shutes, Miss. Elizabeth W;female;40;0;0;PC 17582;153,4625;C125;S;3;;New York, NY / Greenwich CT
+1;1;Silverthorne, Mr. Spencer Victor;male;35;0;0;PC 17475;26,2875;E24;S;5;;St Louis, MO
+1;0;Silvey, Mr. William Baird;male;50;1;0;13507;55,9;E44;S;;;Duluth, MN
+1;1;Silvey, Mrs. William Baird (Alice Munger);female;39;1;0;13507;55,9;E44;S;11;;Duluth, MN
+1;1;Simonius-Blumer, Col. Oberst Alfons;male;56;0;0;13213;35,5;A26;C;3;;Basel, Switzerland
+1;1;Sloper, Mr. William Thompson;male;28;0;0;113788;35,5;A6;S;7;;New Britain, CT
+1;0;Smart, Mr. John Montgomery;male;56;0;0;113792;26,55;;S;;;New York, NY
+1;0;Smith, Mr. James Clinch;male;56;0;0;17764;30,6958;A7;C;;;St James, Long Island, NY
+1;0;Smith, Mr. Lucien Philip;male;24;1;0;13695;60;C31;S;;;Huntington, WV
+1;0;Smith, Mr. Richard William;male;;0;0;113056;26;A19;S;;;Streatham, Surrey
+1;1;Smith, Mrs. Lucien Philip (Mary Eloise Hughes);female;18;1;0;13695;60;C31;S;6;;Huntington, WV
+1;1;Snyder, Mr. John Pillsbury;male;24;1;0;21228;82,2667;B45;S;7;;Minneapolis, MN
+1;1;Snyder, Mrs. John Pillsbury (Nelle Stevenson);female;23;1;0;21228;82,2667;B45;S;7;;Minneapolis, MN
+1;1;Spedden, Master. Robert Douglas;male;6;0;2;16966;134,5;E34;C;3;;Tuxedo Park, NY
+1;1;Spedden, Mr. Frederic Oakley;male;45;1;1;16966;134,5;E34;C;3;;Tuxedo Park, NY
+1;1;Spedden, Mrs. Frederic Oakley (Margaretta Corning Stone);female;40;1;1;16966;134,5;E34;C;3;;Tuxedo Park, NY
+1;0;Spencer, Mr. William Augustus;male;57;1;0;PC 17569;146,5208;B78;C;;;Paris, France
+1;1;Spencer, Mrs. William Augustus (Marie Eugenie);female;;1;0;PC 17569;146,5208;B78;C;6;;Paris, France
+1;1;Stahelin-Maeglin, Dr. Max;male;32;0;0;13214;30,5;B50;C;3;;Basel, Switzerland
+1;0;Stead, Mr. William Thomas;male;62;0;0;113514;26,55;C87;S;;;Wimbledon Park, London / Hayling Island, Hants
+1;1;Stengel, Mr. Charles Emil Henry;male;54;1;0;11778;55,4417;C116;C;1;;Newark, NJ
+1;1;Stengel, Mrs. Charles Emil Henry (Annie May Morris);female;43;1;0;11778;55,4417;C116;C;5;;Newark, NJ
+1;1;Stephenson, Mrs. Walter Bertram (Martha Eustis);female;52;1;0;36947;78,2667;D20;C;4;;Haverford, PA
+1;0;Stewart, Mr. Albert A;male;;0;0;PC 17605;27,7208;;C;;;Gallipolis, Ohio / ? Paris / New York
+1;1;Stone, Mrs. George Nelson (Martha Evelyn);female;62;0;0;113572;80;B28;;6;;Cincinatti, OH
+1;0;Straus, Mr. Isidor;male;67;1;0;PC 17483;221,7792;C55 C57;S;;96;New York, NY
+1;0;Straus, Mrs. Isidor (Rosalie Ida Blun);female;63;1;0;PC 17483;221,7792;C55 C57;S;;;New York, NY
+1;0;Sutton, Mr. Frederick;male;61;0;0;36963;32,3208;D50;S;;46;Haddenfield, NJ
+1;1;Swift, Mrs. Frederick Joel (Margaret Welles Barron);female;48;0;0;17466;25,9292;D17;S;8;;Brooklyn, NY
+1;1;Taussig, Miss. Ruth;female;18;0;2;110413;79,65;E68;S;8;;New York, NY
+1;0;Taussig, Mr. Emil;male;52;1;1;110413;79,65;E67;S;;;New York, NY
+1;1;Taussig, Mrs. Emil (Tillie Mandelbaum);female;39;1;1;110413;79,65;E67;S;8;;New York, NY
+1;1;Taylor, Mr. Elmer Zebley;male;48;1;0;19996;52;C126;S;5 7;;London /  East Orange, NJ
+1;1;Taylor, Mrs. Elmer Zebley (Juliet Cummins Wright);female;;1;0;19996;52;C126;S;5 7;;London /  East Orange, NJ
+1;0;Thayer, Mr. John Borland;male;49;1;1;17421;110,8833;C68;C;;;Haverford, PA
+1;1;Thayer, Mr. John Borland Jr;male;17;0;2;17421;110,8833;C70;C;B;;Haverford, PA
+1;1;Thayer, Mrs. John Borland (Marian Longstreth Morris);female;39;1;1;17421;110,8833;C68;C;4;;Haverford, PA
+1;1;Thorne, Mrs. Gertrude Maybelle;female;;0;0;PC 17585;79,2;;C;D;;New York, NY
+1;1;Tucker, Mr. Gilbert Milligan Jr;male;31;0;0;2543;28,5375;C53;C;7;;Albany, NY
+1;0;Uruchurtu, Don. Manuel E;male;40;0;0;PC 17601;27,7208;;C;;;Mexico City, Mexico
+1;0;Van der hoef, Mr. Wyckoff;male;61;0;0;111240;33,5;B19;S;;245;Brooklyn, NY
+1;0;Walker, Mr. William Anderson;male;47;0;0;36967;34,0208;D46;S;;;East Orange, NJ
+1;1;Ward, Miss. Anna;female;35;0;0;PC 17755;512,3292;;C;3;;
+1;0;Warren, Mr. Frank Manley;male;64;1;0;110813;75,25;D37;C;;;Portland, OR
+1;1;Warren, Mrs. Frank Manley (Anna Sophia Atkinson);female;60;1;0;110813;75,25;D37;C;5;;Portland, OR
+1;0;Weir, Col. John;male;60;0;0;113800;26,55;;S;;;England Salt Lake City, Utah
+1;0;White, Mr. Percival Wayland;male;54;0;1;35281;77,2875;D26;S;;;Brunswick, ME
+1;0;White, Mr. Richard Frasar;male;21;0;1;35281;77,2875;D26;S;;169;Brunswick, ME
+1;1;White, Mrs. John Stuart (Ella Holmes);female;55;0;0;PC 17760;135,6333;C32;C;8;;New York, NY / Briarcliff Manor NY
+1;1;Wick, Miss. Mary Natalie;female;31;0;2;36928;164,8667;C7;S;8;;Youngstown, OH
+1;0;Wick, Mr. George Dennick;male;57;1;1;36928;164,8667;;S;;;Youngstown, OH
+1;1;Wick, Mrs. George Dennick (Mary Hitchcock);female;45;1;1;36928;164,8667;;S;8;;Youngstown, OH
+1;0;Widener, Mr. George Dunton;male;50;1;1;113503;211,5;C80;C;;;Elkins Park, PA
+1;0;Widener, Mr. Harry Elkins;male;27;0;2;113503;211,5;C82;C;;;Elkins Park, PA
+1;1;Widener, Mrs. George Dunton (Eleanor Elkins);female;50;1;1;113503;211,5;C80;C;4;;Elkins Park, PA
+1;1;Willard, Miss. Constance;female;21;0;0;113795;26,55;;S;8 10;;Duluth, MN
+1;0;Williams, Mr. Charles Duane;male;51;0;1;PC 17597;61,3792;;C;;;Geneva, Switzerland / Radnor, PA
+1;1;Williams, Mr. Richard Norris II;male;21;0;1;PC 17597;61,3792;;C;A;;Geneva, Switzerland / Radnor, PA
+1;0;Williams-Lambert, Mr. Fletcher Fellows;male;;0;0;113510;35;C128;S;;;London, England
+1;1;Wilson, Miss. Helen Alice;female;31;0;0;16966;134,5;E39 E41;C;3;;
+1;1;Woolner, Mr. Hugh;male;;0;0;19947;35,5;C52;S;D;;London, England
+1;0;Wright, Mr. George;male;62;0;0;113807;26,55;;S;;;Halifax, NS
+1;1;Young, Miss. Marie Grice;female;36;0;0;PC 17760;135,6333;C32;C;8;;New York, NY / Washington, DC
+2;0;Abelson, Mr. Samuel;male;30;1;0;P/PP 3381;24;;C;;;Russia New York, NY
+2;1;Abelson, Mrs. Samuel (Hannah Wizosky);female;28;1;0;P/PP 3381;24;;C;10;;Russia New York, NY
+2;0;Aldworth, Mr. Charles Augustus;male;30;0;0;248744;13;;S;;;Bryn Mawr, PA, USA
+2;0;Andrew, Mr. Edgardo Samuel;male;18;0;0;231945;11,5;;S;;;Buenos Aires, Argentina / New Jersey, NJ
+2;0;Andrew, Mr. Frank Thomas;male;25;0;0;C.A. 34050;10,5;;S;;;Cornwall, England Houghton, MI
+2;0;Angle, Mr. William A;male;34;1;0;226875;26;;S;;;Warwick, England
+2;1;"Angle, Mrs. William A (Florence ""Mary"" Agnes Hughes)";female;36;1;0;226875;26;;S;11;;Warwick, England
+2;0;Ashby, Mr. John;male;57;0;0;244346;13;;S;;;West Hoboken, NJ
+2;0;Bailey, Mr. Percy Andrew;male;18;0;0;29108;11,5;;S;;;Penzance, Cornwall / Akron, OH
+2;0;Baimbrigge, Mr. Charles Robert;male;23;0;0;C.A. 31030;10,5;;S;;;Guernsey
+2;1;Ball, Mrs. (Ada E Hall);female;36;0;0;28551;13;D;S;10;;Bristol, Avon / Jacksonville, FL
+2;0;Banfield, Mr. Frederick James;male;28;0;0;C.A./SOTON 34068;10,5;;S;;;Plymouth, Dorset / Houghton, MI
+2;0;Bateman, Rev. Robert James;male;51;0;0;S.O.P. 1166;12,525;;S;;174;Jacksonville, FL
+2;1;Beane, Mr. Edward;male;32;1;0;2908;26;;S;13;;Norwich / New York, NY
+2;1;Beane, Mrs. Edward (Ethel Clarke);female;19;1;0;2908;26;;S;13;;Norwich / New York, NY
+2;0;Beauchamp, Mr. Henry James;male;28;0;0;244358;26;;S;;;England
+2;1;Becker, Master. Richard F;male;1;2;1;230136;39;F4;S;11;;Guntur, India / Benton Harbour, MI
+2;1;Becker, Miss. Marion Louise;female;4;2;1;230136;39;F4;S;11;;Guntur, India / Benton Harbour, MI
+2;1;Becker, Miss. Ruth Elizabeth;female;12;2;1;230136;39;F4;S;13;;Guntur, India / Benton Harbour, MI
+2;1;Becker, Mrs. Allen Oliver (Nellie E Baumgardner);female;36;0;3;230136;39;F4;S;11;;Guntur, India / Benton Harbour, MI
+2;1;Beesley, Mr. Lawrence;male;34;0;0;248698;13;D56;S;13;;London
+2;1;Bentham, Miss. Lilian W;female;19;0;0;28404;13;;S;12;;Rochester, NY
+2;0;Berriman, Mr. William John;male;23;0;0;28425;13;;S;;;St Ives, Cornwall / Calumet, MI
+2;0;Botsford, Mr. William Hull;male;26;0;0;237670;13;;S;;;Elmira, NY / Orange, NJ
+2;0;Bowenur, Mr. Solomon;male;42;0;0;211535;13;;S;;;London
+2;0;Bracken, Mr. James H;male;27;0;0;220367;13;;S;;;Lake Arthur, Chavez County, NM
+2;1;"Brown, Miss. Amelia ""Mildred""";female;24;0;0;248733;13;F33;S;11;;London / Montreal, PQ
+2;1;Brown, Miss. Edith Eileen;female;15;0;2;29750;39;;S;14;;Cape Town, South Africa / Seattle, WA
+2;0;Brown, Mr. Thomas William Solomon;male;60;1;1;29750;39;;S;;;Cape Town, South Africa / Seattle, WA
+2;1;Brown, Mrs. Thomas William Solomon (Elizabeth Catherine Ford);female;40;1;1;29750;39;;S;14;;Cape Town, South Africa / Seattle, WA
+2;1;Bryhl, Miss. Dagmar Jenny Ingeborg ;female;20;1;0;236853;26;;S;12;;Skara, Sweden / Rockford, IL
+2;0;Bryhl, Mr. Kurt Arnold Gottfrid;male;25;1;0;236853;26;;S;;;Skara, Sweden / Rockford, IL
+2;1;Buss, Miss. Kate;female;36;0;0;27849;13;;S;9;;Sittingbourne, England / San Diego, CA
+2;0;Butler, Mr. Reginald Fenton;male;25;0;0;234686;13;;S;;97;Southsea, Hants
+2;0;Byles, Rev. Thomas Roussel Davids;male;42;0;0;244310;13;;S;;;London
+2;1;Bystrom, Mrs. (Karolina);female;42;0;0;236852;13;;S;;;New York, NY
+2;1;Caldwell, Master. Alden Gates;male;0,8333;0;2;248738;29;;S;13;;Bangkok, Thailand / Roseville, IL
+2;1;Caldwell, Mr. Albert Francis;male;26;1;1;248738;29;;S;13;;Bangkok, Thailand / Roseville, IL
+2;1;Caldwell, Mrs. Albert Francis (Sylvia Mae Harbaugh);female;22;1;1;248738;29;;S;13;;Bangkok, Thailand / Roseville, IL
+2;1;Cameron, Miss. Clear Annie;female;35;0;0;F.C.C. 13528;21;;S;14;;Mamaroneck, NY
+2;0;Campbell, Mr. William;male;;0;0;239853;0;;S;;;Belfast
+2;0;Carbines, Mr. William;male;19;0;0;28424;13;;S;;18;St Ives, Cornwall / Calumet, MI
+2;0;Carter, Mrs. Ernest Courtenay (Lilian Hughes);female;44;1;0;244252;26;;S;;;London
+2;0;Carter, Rev. Ernest Courtenay;male;54;1;0;244252;26;;S;;;London
+2;0;Chapman, Mr. Charles Henry;male;52;0;0;248731;13,5;;S;;130;Bronx, NY
+2;0;Chapman, Mr. John Henry;male;37;1;0;SC/AH 29037;26;;S;;17;Cornwall / Spokane, WA
+2;0;Chapman, Mrs. John Henry (Sara Elizabeth Lawry);female;29;1;0;SC/AH 29037;26;;S;;;Cornwall / Spokane, WA
+2;1;Christy, Miss. Julie Rachel;female;25;1;1;237789;30;;S;12;;London
+2;1;Christy, Mrs. (Alice Frances);female;45;0;2;237789;30;;S;12;;London
+2;0;Clarke, Mr. Charles Valentine;male;29;1;0;2003;26;;S;;;England / San Francisco, CA
+2;1;Clarke, Mrs. Charles V (Ada Maria Winfield);female;28;1;0;2003;26;;S;14;;England / San Francisco, CA
+2;0;Coleridge, Mr. Reginald Charles;male;29;0;0;W./C. 14263;10,5;;S;;;Hartford, Huntingdonshire
+2;0;Collander, Mr. Erik Gustaf;male;28;0;0;248740;13;;S;;;Helsinki, Finland Ashtabula, Ohio
+2;1;Collett, Mr. Sidney C Stuart;male;24;0;0;28034;10,5;;S;9;;London / Fort Byron, NY
+2;1;"Collyer, Miss. Marjorie ""Lottie""";female;8;0;2;C.A. 31921;26,25;;S;14;;Bishopstoke, Hants / Fayette Valley, ID
+2;0;Collyer, Mr. Harvey;male;31;1;1;C.A. 31921;26,25;;S;;;Bishopstoke, Hants / Fayette Valley, ID
+2;1;Collyer, Mrs. Harvey (Charlotte Annie Tate);female;31;1;1;C.A. 31921;26,25;;S;14;;Bishopstoke, Hants / Fayette Valley, ID
+2;1;Cook, Mrs. (Selena Rogers);female;22;0;0;W./C. 14266;10,5;F33;S;14;;Pennsylvania
+2;0;Corbett, Mrs. Walter H (Irene Colvin);female;30;0;0;237249;13;;S;;;Provo, UT
+2;0;Corey, Mrs. Percy C (Mary Phyllis Elizabeth Miller);female;;0;0;F.C.C. 13534;21;;S;;;Upper Burma, India Pittsburgh, PA
+2;0;"Cotterill, Mr. Henry ""Harry""";male;21;0;0;29107;11,5;;S;;;Penzance, Cornwall / Akron, OH
+2;0;Cunningham, Mr. Alfred Fleming;male;;0;0;239853;0;;S;;;Belfast
+2;1;Davies, Master. John Morgan Jr;male;8;1;1;C.A. 33112;36,75;;S;14;;St Ives, Cornwall / Hancock, MI
+2;0;Davies, Mr. Charles Henry;male;18;0;0;S.O.C. 14879;73,5;;S;;;Lyndhurst, England
+2;1;Davies, Mrs. John Morgan (Elizabeth Agnes Mary White) ;female;48;0;2;C.A. 33112;36,75;;S;14;;St Ives, Cornwall / Hancock, MI
+2;1;Davis, Miss. Mary;female;28;0;0;237668;13;;S;13;;London / Staten Island, NY
+2;0;de Brito, Mr. Jose Joaquim;male;32;0;0;244360;13;;S;;;Portugal / Sau Paulo, Brazil
+2;0;Deacon, Mr. Percy William;male;17;0;0;S.O.C. 14879;73,5;;S;;;
+2;0;del Carlo, Mr. Sebastiano;male;29;1;0;SC/PARIS 2167;27,7208;;C;;295;Lucca, Italy / California
+2;1;del Carlo, Mrs. Sebastiano (Argenia Genovesi);female;24;1;0;SC/PARIS 2167;27,7208;;C;12;;Lucca, Italy / California
+2;0;Denbury, Mr. Herbert;male;25;0;0;C.A. 31029;31,5;;S;;;Guernsey / Elizabeth, NJ
+2;0;Dibden, Mr. William;male;18;0;0;S.O.C. 14879;73,5;;S;;;New Forest, England
+2;1;Doling, Miss. Elsie;female;18;0;1;231919;23;;S;;;Southampton
+2;1;Doling, Mrs. John T (Ada Julia Bone);female;34;0;1;231919;23;;S;;;Southampton
+2;0;Downton, Mr. William James;male;54;0;0;28403;26;;S;;;Holley, NY
+2;1;Drew, Master. Marshall Brines;male;8;0;2;28220;32,5;;S;10;;Greenport, NY
+2;0;Drew, Mr. James Vivian;male;42;1;1;28220;32,5;;S;;;Greenport, NY
+2;1;Drew, Mrs. James Vivian (Lulu Thorne Christian);female;34;1;1;28220;32,5;;S;10;;Greenport, NY
+2;1;Duran y More, Miss. Asuncion;female;27;1;0;SC/PARIS 2149;13,8583;;C;12;;Barcelona, Spain / Havana, Cuba
+2;1;Duran y More, Miss. Florentina;female;30;1;0;SC/PARIS 2148;13,8583;;C;12;;Barcelona, Spain / Havana, Cuba
+2;0;Eitemiller, Mr. George Floyd;male;23;0;0;29751;13;;S;;;England / Detroit, MI
+2;0;Enander, Mr. Ingvar;male;21;0;0;236854;13;;S;;;Goteborg, Sweden / Rockford, IL
+2;0;Fahlstrom, Mr. Arne Jonas;male;18;0;0;236171;13;;S;;;Oslo, Norway Bayonne, NJ
+2;0;Faunthorpe, Mr. Harry;male;40;1;0;2926;26;;S;;286;England / Philadelphia, PA
+2;1;Faunthorpe, Mrs. Lizzie (Elizabeth Anne Wilkinson);female;29;1;0;2926;26;;S;16;;
+2;0;Fillbrook, Mr. Joseph Charles;male;18;0;0;C.A. 15185;10,5;;S;;;Cornwall / Houghton, MI
+2;0;Fox, Mr. Stanley Hubert;male;36;0;0;229236;13;;S;;236;Rochester, NY
+2;0;"Frost, Mr. Anthony Wood ""Archie""";male;;0;0;239854;0;;S;;;Belfast
+2;0;Funk, Miss. Annie Clemmer;female;38;0;0;237671;13;;S;;;Janjgir, India / Pennsylvania
+2;0;Fynney, Mr. Joseph J;male;35;0;0;239865;26;;S;;322;Liverpool / Montreal, PQ
+2;0;Gale, Mr. Harry;male;38;1;0;28664;21;;S;;;Cornwall / Clear Creek, CO
+2;0;Gale, Mr. Shadrach;male;34;1;0;28664;21;;S;;;Cornwall / Clear Creek, CO
+2;1;Garside, Miss. Ethel;female;34;0;0;243880;13;;S;12;;Brooklyn, NY
+2;0;Gaskell, Mr. Alfred;male;16;0;0;239865;26;;S;;;Liverpool / Montreal, PQ
+2;0;Gavey, Mr. Lawrence;male;26;0;0;31028;10,5;;S;;;Guernsey / Elizabeth, NJ
+2;0;Gilbert, Mr. William;male;47;0;0;C.A. 30769;10,5;;S;;;Cornwall
+2;0;Giles, Mr. Edgar;male;21;1;0;28133;11,5;;S;;;Cornwall / Camden, NJ
+2;0;Giles, Mr. Frederick Edward;male;21;1;0;28134;11,5;;S;;;Cornwall / Camden, NJ
+2;0;Giles, Mr. Ralph;male;24;0;0;248726;13,5;;S;;297;West Kensington, London
+2;0;Gill, Mr. John William;male;24;0;0;233866;13;;S;;155;Clevedon, England
+2;0;Gillespie, Mr. William Henry;male;34;0;0;12233;13;;S;;;Vancouver, BC
+2;0;Givard, Mr. Hans Kristensen;male;30;0;0;250646;13;;S;;305;
+2;0;Greenberg, Mr. Samuel;male;52;0;0;250647;13;;S;;19;Bronx, NY
+2;0;Hale, Mr. Reginald;male;30;0;0;250653;13;;S;;75;Auburn, NY
+2;1;Hamalainen, Master. Viljo;male;0,6667;1;1;250649;14,5;;S;4;;Detroit, MI
+2;1;Hamalainen, Mrs. William (Anna);female;24;0;2;250649;14,5;;S;4;;Detroit, MI
+2;0;Harbeck, Mr. William H;male;44;0;0;248746;13;;S;;35;Seattle, WA / Toledo, OH
+2;1;"Harper, Miss. Annie Jessie ""Nina""";female;6;0;1;248727;33;;S;11;;Denmark Hill, Surrey / Chicago
+2;0;Harper, Rev. John;male;28;0;1;248727;33;;S;;;Denmark Hill, Surrey / Chicago
+2;1;Harris, Mr. George;male;62;0;0;S.W./PP 752;10,5;;S;15;;London
+2;0;Harris, Mr. Walter;male;30;0;0;W/C 14208;10,5;;S;;;Walthamstow, England
+2;1;Hart, Miss. Eva Miriam;female;7;0;2;F.C.C. 13529;26,25;;S;14;;Ilford, Essex / Winnipeg, MB
+2;0;Hart, Mr. Benjamin;male;43;1;1;F.C.C. 13529;26,25;;S;;;Ilford, Essex / Winnipeg, MB
+2;1;Hart, Mrs. Benjamin (Esther Ada Bloomfield);female;45;1;1;F.C.C. 13529;26,25;;S;14;;Ilford, Essex / Winnipeg, MB
+2;1;Herman, Miss. Alice;female;24;1;2;220845;65;;S;9;;Somerset / Bernardsville, NJ
+2;1;Herman, Miss. Kate;female;24;1;2;220845;65;;S;9;;Somerset / Bernardsville, NJ
+2;0;Herman, Mr. Samuel;male;49;1;2;220845;65;;S;;;Somerset / Bernardsville, NJ
+2;1;Herman, Mrs. Samuel (Jane Laver);female;48;1;2;220845;65;;S;9;;Somerset / Bernardsville, NJ
+2;1;Hewlett, Mrs. (Mary D Kingcome) ;female;55;0;0;248706;16;;S;13;;India / Rapid City, SD
+2;0;Hickman, Mr. Leonard Mark;male;24;2;0;S.O.C. 14879;73,5;;S;;;West Hampstead, London / Neepawa, MB
+2;0;Hickman, Mr. Lewis;male;32;2;0;S.O.C. 14879;73,5;;S;;256;West Hampstead, London / Neepawa, MB
+2;0;Hickman, Mr. Stanley George;male;21;2;0;S.O.C. 14879;73,5;;S;;;West Hampstead, London / Neepawa, MB
+2;0;Hiltunen, Miss. Marta;female;18;1;1;250650;13;;S;;;Kontiolahti, Finland / Detroit, MI
+2;1;"Hocking, Miss. Ellen ""Nellie""";female;20;2;1;29105;23;;S;4;;Cornwall / Akron, OH
+2;0;Hocking, Mr. Richard George;male;23;2;1;29104;11,5;;S;;;Cornwall / Akron, OH
+2;0;Hocking, Mr. Samuel James Metcalfe;male;36;0;0;242963;13;;S;;;Devonport, England
+2;1;Hocking, Mrs. Elizabeth (Eliza Needs);female;54;1;3;29105;23;;S;4;;Cornwall / Akron, OH
+2;0;Hodges, Mr. Henry Price;male;50;0;0;250643;13;;S;;149;Southampton
+2;0;Hold, Mr. Stephen;male;44;1;0;26707;26;;S;;;England / Sacramento, CA
+2;1;Hold, Mrs. Stephen (Annie Margaret Hill);female;29;1;0;26707;26;;S;10;;England / Sacramento, CA
+2;0;Hood, Mr. Ambrose Jr;male;21;0;0;S.O.C. 14879;73,5;;S;;;New Forest, England
+2;1;Hosono, Mr. Masabumi;male;42;0;0;237798;13;;S;10;;Tokyo, Japan
+2;0;Howard, Mr. Benjamin;male;63;1;0;24065;26;;S;;;Swindon, England
+2;0;Howard, Mrs. Benjamin (Ellen Truelove Arman);female;60;1;0;24065;26;;S;;;Swindon, England
+2;0;Hunt, Mr. George Henry;male;33;0;0;SCO/W 1585;12,275;;S;;;Philadelphia, PA
+2;1;Ilett, Miss. Bertha;female;17;0;0;SO/C 14885;10,5;;S;;;Guernsey
+2;0;Jacobsohn, Mr. Sidney Samuel;male;42;1;0;243847;27;;S;;;London
+2;1;Jacobsohn, Mrs. Sidney Samuel (Amy Frances Christy);female;24;2;1;243847;27;;S;12;;London
+2;0;Jarvis, Mr. John Denzil;male;47;0;0;237565;15;;S;;;North Evington, England
+2;0;Jefferys, Mr. Clifford Thomas;male;24;2;0;C.A. 31029;31,5;;S;;;Guernsey / Elizabeth, NJ
+2;0;Jefferys, Mr. Ernest Wilfred;male;22;2;0;C.A. 31029;31,5;;S;;;Guernsey / Elizabeth, NJ
+2;0;Jenkin, Mr. Stephen Curnow;male;32;0;0;C.A. 33111;10,5;;S;;;St Ives, Cornwall / Houghton, MI
+2;1;Jerwan, Mrs. Amin S (Marie Marthe Thuillard);female;23;0;0;SC/AH Basle 541;13,7917;D;C;11;;New York, NY
+2;0;Kantor, Mr. Sinai;male;34;1;0;244367;26;;S;;283;Moscow / Bronx, NY
+2;1;Kantor, Mrs. Sinai (Miriam Sternin);female;24;1;0;244367;26;;S;12;;Moscow / Bronx, NY
+2;0;Karnes, Mrs. J Frank (Claire Bennett);female;22;0;0;F.C.C. 13534;21;;S;;;India / Pittsburgh, PA
+2;1;Keane, Miss. Nora A;female;;0;0;226593;12,35;E101;Q;10;;Harrisburg, PA
+2;0;Keane, Mr. Daniel;male;35;0;0;233734;12,35;;Q;;;
+2;1;"Kelly, Mrs. Florence ""Fannie""";female;45;0;0;223596;13,5;;S;9;;London / New York, NY
+2;0;Kirkland, Rev. Charles Leonard;male;57;0;0;219533;12,35;;Q;;;Glasgow / Bangor, ME
+2;0;Knight, Mr. Robert J;male;;0;0;239855;0;;S;;;Belfast
+2;0;Kvillner, Mr. Johan Henrik Johannesson;male;31;0;0;C.A. 18723;10,5;;S;;165;Sweden / Arlington, NJ
+2;0;Lahtinen, Mrs. William (Anna Sylfven);female;26;1;1;250651;26;;S;;;Minneapolis, MN
+2;0;Lahtinen, Rev. William;male;30;1;1;250651;26;;S;;;Minneapolis, MN
+2;0;Lamb, Mr. John Joseph;male;;0;0;240261;10,7083;;Q;;;
+2;1;Laroche, Miss. Louise;female;1;1;2;SC/Paris 2123;41,5792;;C;14;;Paris / Haiti
+2;1;Laroche, Miss. Simonne Marie Anne Andree;female;3;1;2;SC/Paris 2123;41,5792;;C;14;;Paris / Haiti
+2;0;Laroche, Mr. Joseph Philippe Lemercier;male;25;1;2;SC/Paris 2123;41,5792;;C;;;Paris / Haiti
+2;1;Laroche, Mrs. Joseph (Juliette Marie Louise Lafargue);female;22;1;2;SC/Paris 2123;41,5792;;C;14;;Paris / Haiti
+2;1;Lehmann, Miss. Bertha;female;17;0;0;SC 1748;12;;C;12;;Berne, Switzerland / Central City, IA
+2;1;Leitch, Miss. Jessie Wills;female;;0;0;248727;33;;S;11;;London / Chicago, IL
+2;1;Lemore, Mrs. (Amelia Milley);female;34;0;0;C.A. 34260;10,5;F33;S;14;;Chicago, IL
+2;0;Levy, Mr. Rene Jacques;male;36;0;0;SC/Paris 2163;12,875;D;C;;;Montreal, PQ
+2;0;Leyson, Mr. Robert William Norman;male;24;0;0;C.A. 29566;10,5;;S;;108;
+2;0;Lingane, Mr. John;male;61;0;0;235509;12,35;;Q;;;
+2;0;Louch, Mr. Charles Alexander;male;50;1;0;SC/AH 3085;26;;S;;121;Weston-Super-Mare, Somerset
+2;1;Louch, Mrs. Charles Alexander (Alice Adelaide Slow);female;42;1;0;SC/AH 3085;26;;S;;;Weston-Super-Mare, Somerset
+2;0;Mack, Mrs. (Mary);female;57;0;0;S.O./P.P. 3;10,5;E77;S;;52;Southampton / New York, NY
+2;0;Malachard, Mr. Noel;male;;0;0;237735;15,0458;D;C;;;Paris
+2;1;Mallet, Master. Andre;male;1;0;2;S.C./PARIS 2079;37,0042;;C;10;;Paris / Montreal, PQ
+2;0;Mallet, Mr. Albert;male;31;1;1;S.C./PARIS 2079;37,0042;;C;;;Paris / Montreal, PQ
+2;1;Mallet, Mrs. Albert (Antoinette Magnin);female;24;1;1;S.C./PARIS 2079;37,0042;;C;10;;Paris / Montreal, PQ
+2;0;Mangiavacchi, Mr. Serafino Emilio;male;;0;0;SC/A.3 2861;15,5792;;C;;;New York, NY
+2;0;Matthews, Mr. William John;male;30;0;0;28228;13;;S;;;St Austall, Cornwall
+2;0;Maybery, Mr. Frank Hubert;male;40;0;0;239059;16;;S;;;Weston-Super-Mare / Moose Jaw, SK
+2;0;McCrae, Mr. Arthur Gordon;male;32;0;0;237216;13,5;;S;;209;Sydney, Australia
+2;0;McCrie, Mr. James Matthew;male;30;0;0;233478;13;;S;;;Sarnia, ON
+2;0;McKane, Mr. Peter David;male;46;0;0;28403;26;;S;;;Rochester, NY
+2;1;Mellinger, Miss. Madeleine Violet;female;13;0;1;250644;19,5;;S;14;;England / Bennington, VT
+2;1;Mellinger, Mrs. (Elizabeth Anne Maidment);female;41;0;1;250644;19,5;;S;14;;England / Bennington, VT
+2;1;Mellors, Mr. William John;male;19;0;0;SW/PP 751;10,5;;S;B;;Chelsea, London
+2;0;Meyer, Mr. August;male;39;0;0;248723;13;;S;;;Harrow-on-the-Hill, Middlesex
+2;0;Milling, Mr. Jacob Christian;male;48;0;0;234360;13;;S;;271;Copenhagen, Denmark
+2;0;Mitchell, Mr. Henry Michael;male;70;0;0;C.A. 24580;10,5;;S;;;Guernsey / Montclair, NJ and/or Toledo, Ohio
+2;0;Montvila, Rev. Juozas;male;27;0;0;211536;13;;S;;;Worcester, MA
+2;0;Moraweck, Dr. Ernest;male;54;0;0;29011;14;;S;;;Frankfort, KY
+2;0;"Morley, Mr. Henry Samuel (""Mr Henry Marshall"")";male;39;0;0;250655;26;;S;;;
+2;0;Mudd, Mr. Thomas Charles;male;16;0;0;S.O./P.P. 3;10,5;;S;;;Halesworth, England
+2;0;Myles, Mr. Thomas Francis;male;62;0;0;240276;9,6875;;Q;;;Cambridge, MA
+2;0;Nasser, Mr. Nicholas;male;32,5;1;0;237736;30,0708;;C;;43;New York, NY
+2;1;Nasser, Mrs. Nicholas (Adele Achem);female;14;1;0;237736;30,0708;;C;;;New York, NY
+2;1;Navratil, Master. Edmond Roger;male;2;1;1;230080;26;F2;S;D;;Nice, France
+2;1;Navratil, Master. Michel M;male;3;1;1;230080;26;F2;S;D;;Nice, France
+2;0;"Navratil, Mr. Michel (""Louis M Hoffman"")";male;36,5;0;2;230080;26;F2;S;;15;Nice, France
+2;0;Nesson, Mr. Israel;male;26;0;0;244368;13;F2;S;;;Boston, MA
+2;0;Nicholls, Mr. Joseph Charles;male;19;1;1;C.A. 33112;36,75;;S;;101;Cornwall / Hancock, MI
+2;0;Norman, Mr. Robert Douglas;male;28;0;0;218629;13,5;;S;;287;Glasgow
+2;1;"Nourney, Mr. Alfred (""Baron von Drachstedt"")";male;20;0;0;SC/PARIS 2166;13,8625;D38;C;7;;Cologne, Germany
+2;1;Nye, Mrs. (Elizabeth Ramell);female;29;0;0;C.A. 29395;10,5;F33;S;11;;Folkstone, Kent / New York, NY
+2;0;Otter, Mr. Richard;male;39;0;0;28213;13;;S;;;Middleburg Heights, OH
+2;1;Oxenham, Mr. Percy Thomas;male;22;0;0;W./C. 14260;10,5;;S;13;;Pondersend, England / New Durham, NJ
+2;1;Padro y Manent, Mr. Julian;male;;0;0;SC/PARIS 2146;13,8625;;C;9;;Spain / Havana, Cuba
+2;0;Pain, Dr. Alfred;male;23;0;0;244278;10,5;;S;;;Hamilton, ON
+2;1;Pallas y Castello, Mr. Emilio;male;29;0;0;SC/PARIS 2147;13,8583;;C;9;;Spain / Havana, Cuba
+2;0;Parker, Mr. Clifford Richard;male;28;0;0;SC 14888;10,5;;S;;;St Andrews, Guernsey
+2;0;"Parkes, Mr. Francis ""Frank""";male;;0;0;239853;0;;S;;;Belfast
+2;1;Parrish, Mrs. (Lutie Davis);female;50;0;1;230433;26;;S;12;;Woodford County, KY
+2;0;Pengelly, Mr. Frederick William;male;19;0;0;28665;10,5;;S;;;Gunnislake, England / Butte, MT
+2;0;Pernot, Mr. Rene;male;;0;0;SC/PARIS 2131;15,05;;C;;;
+2;0;Peruschitz, Rev. Joseph Maria;male;41;0;0;237393;13;;S;;;
+2;1;Phillips, Miss. Alice Frances Louisa;female;21;0;1;S.O./P.P. 2;21;;S;12;;Ilfracombe, Devon
+2;1;"Phillips, Miss. Kate Florence (""Mrs Kate Louise Phillips Marshall"")";female;19;0;0;250655;26;;S;11;;Worcester, England
+2;0;Phillips, Mr. Escott Robert;male;43;0;1;S.O./P.P. 2;21;;S;;;Ilfracombe, Devon
+2;1;Pinsky, Mrs. (Rosa);female;32;0;0;234604;13;;S;9;;Russia
+2;0;Ponesell, Mr. Martin;male;34;0;0;250647;13;;S;;;Denmark / New York, NY
+2;1;Portaluppi, Mr. Emilio Ilario Giuseppe;male;30;0;0;C.A. 34644;12,7375;;C;14;;Milford, NH
+2;0;Pulbaum, Mr. Franz;male;27;0;0;SC/PARIS 2168;15,0333;;C;;;Paris
+2;1;Quick, Miss. Phyllis May;female;2;1;1;26360;26;;S;11;;Plymouth, Devon / Detroit, MI
+2;1;Quick, Miss. Winifred Vera;female;8;1;1;26360;26;;S;11;;Plymouth, Devon / Detroit, MI
+2;1;Quick, Mrs. Frederick Charles (Jane Richards);female;33;0;2;26360;26;;S;11;;Plymouth, Devon / Detroit, MI
+2;0;Reeves, Mr. David;male;36;0;0;C.A. 17248;10,5;;S;;;Brighton, Sussex
+2;0;Renouf, Mr. Peter Henry;male;34;1;0;31027;21;;S;12;;Elizabeth, NJ
+2;1;Renouf, Mrs. Peter Henry (Lillian Jefferys);female;30;3;0;31027;21;;S;;;Elizabeth, NJ
+2;1;Reynaldo, Ms. Encarnacion;female;28;0;0;230434;13;;S;9;;Spain
+2;0;Richard, Mr. Emile;male;23;0;0;SC/PARIS 2133;15,0458;;C;;;Paris / Montreal, PQ
+2;1;Richards, Master. George Sibley;male;0,8333;1;1;29106;18,75;;S;4;;Cornwall / Akron, OH
+2;1;Richards, Master. William Rowe;male;3;1;1;29106;18,75;;S;4;;Cornwall / Akron, OH
+2;1;Richards, Mrs. Sidney (Emily Hocking);female;24;2;3;29106;18,75;;S;4;;Cornwall / Akron, OH
+2;1;Ridsdale, Miss. Lucy;female;50;0;0;W./C. 14258;10,5;;S;13;;London, England / Marietta, Ohio and Milwaukee, WI
+2;0;Rogers, Mr. Reginald Harry;male;19;0;0;28004;10,5;;S;;;
+2;1;Rugg, Miss. Emily;female;21;0;0;C.A. 31026;10,5;;S;12;;Guernsey / Wilmington, DE
+2;0;Schmidt, Mr. August;male;26;0;0;248659;13;;S;;;Newark, NJ
+2;0;Sedgwick, Mr. Charles Frederick Waddington;male;25;0;0;244361;13;;S;;;Liverpool
+2;0;Sharp, Mr. Percival James R;male;27;0;0;244358;26;;S;;;Hornsey, England
+2;1;Shelley, Mrs. William (Imanita Parrish Hall);female;25;0;1;230433;26;;S;12;;Deer Lodge, MT
+2;1;Silven, Miss. Lyyli Karoliina;female;18;0;2;250652;13;;S;16;;Finland / Minneapolis, MN
+2;1;Sincock, Miss. Maude;female;20;0;0;C.A. 33112;36,75;;S;11;;Cornwall / Hancock, MI
+2;1;Sinkkonen, Miss. Anna;female;30;0;0;250648;13;;S;10;;Finland / Washington, DC
+2;0;Sjostedt, Mr. Ernst Adolf;male;59;0;0;237442;13,5;;S;;;Sault St Marie, ON
+2;1;Slayter, Miss. Hilda Mary;female;30;0;0;234818;12,35;;Q;13;;Halifax, NS
+2;0;Slemen, Mr. Richard James;male;35;0;0;28206;10,5;;S;;;Cornwall
+2;1;Smith, Miss. Marion Elsie;female;40;0;0;31418;13;;S;9;;
+2;0;Sobey, Mr. Samuel James Hayden;male;25;0;0;C.A. 29178;13;;S;;;Cornwall / Houghton, MI
+2;0;Stanton, Mr. Samuel Ward;male;41;0;0;237734;15,0458;;C;;;New York, NY
+2;0;Stokes, Mr. Philip Joseph;male;25;0;0;F.C.C. 13540;10,5;;S;;81;Catford, Kent / Detroit, MI
+2;0;Swane, Mr. George;male;18,5;0;0;248734;13;F;S;;294;
+2;0;Sweet, Mr. George Frederick;male;14;0;0;220845;65;;S;;;Somerset / Bernardsville, NJ
+2;1;Toomey, Miss. Ellen;female;50;0;0;F.C.C. 13531;10,5;;S;9;;Indianapolis, IN
+2;0;Troupiansky, Mr. Moses Aaron;male;23;0;0;233639;13;;S;;;
+2;1;Trout, Mrs. William H (Jessie L);female;28;0;0;240929;12,65;;S;;;Columbus, OH
+2;1;"Troutt, Miss. Edwina Celia ""Winnie""";female;27;0;0;34218;10,5;E101;S;16;;Bath, England / Massachusetts
+2;0;Turpin, Mr. William John Robert;male;29;1;0;11668;21;;S;;;Plymouth, England
+2;0;Turpin, Mrs. William John Robert (Dorothy Ann Wonnacott);female;27;1;0;11668;21;;S;;;Plymouth, England
+2;0;Veal, Mr. James;male;40;0;0;28221;13;;S;;;Barre, Co Washington, VT
+2;1;Walcroft, Miss. Nellie;female;31;0;0;F.C.C. 13528;21;;S;14;;Mamaroneck, NY
+2;0;Ware, Mr. John James;male;30;1;0;CA 31352;21;;S;;;Bristol, England / New Britain, CT
+2;0;Ware, Mr. William Jeffery;male;23;1;0;28666;10,5;;S;;;
+2;1;Ware, Mrs. John James (Florence Louise Long);female;31;0;0;CA 31352;21;;S;10;;Bristol, England / New Britain, CT
+2;0;Watson, Mr. Ennis Hastings;male;;0;0;239856;0;;S;;;Belfast
+2;1;Watt, Miss. Bertha J;female;12;0;0;C.A. 33595;15,75;;S;9;;Aberdeen / Portland, OR
+2;1;"Watt, Mrs. James (Elizabeth ""Bessie"" Inglis Milne)";female;40;0;0;C.A. 33595;15,75;;S;9;;Aberdeen / Portland, OR
+2;1;Webber, Miss. Susan;female;32,5;0;0;27267;13;E101;S;12;;England / Hartford, CT
+2;0;Weisz, Mr. Leopold;male;27;1;0;228414;26;;S;;293;Bromsgrove, England / Montreal, PQ
+2;1;Weisz, Mrs. Leopold (Mathilde Francoise Pede);female;29;1;0;228414;26;;S;10;;Bromsgrove, England / Montreal, PQ
+2;1;Wells, Master. Ralph Lester;male;2;1;1;29103;23;;S;14;;Cornwall / Akron, OH
+2;1;Wells, Miss. Joan;female;4;1;1;29103;23;;S;14;;Cornwall / Akron, OH
+2;1;"Wells, Mrs. Arthur Henry (""Addie"" Dart Trevaskis)";female;29;0;2;29103;23;;S;14;;Cornwall / Akron, OH
+2;1;West, Miss. Barbara J;female;0,9167;1;2;C.A. 34651;27,75;;S;10;;Bournmouth, England
+2;1;West, Miss. Constance Mirium;female;5;1;2;C.A. 34651;27,75;;S;10;;Bournmouth, England
+2;0;West, Mr. Edwy Arthur;male;36;1;2;C.A. 34651;27,75;;S;;;Bournmouth, England
+2;1;West, Mrs. Edwy Arthur (Ada Mary Worth);female;33;1;2;C.A. 34651;27,75;;S;10;;Bournmouth, England
+2;0;Wheadon, Mr. Edward H;male;66;0;0;C.A. 24579;10,5;;S;;;Guernsey, England / Edgewood, RI
+2;0;"Wheeler, Mr. Edwin ""Frederick""";male;;0;0;SC/PARIS 2159;12,875;;S;;;
+2;1;Wilhelms, Mr. Charles;male;31;0;0;244270;13;;S;9;;London, England
+2;1;Williams, Mr. Charles Eugene;male;;0;0;244373;13;;S;14;;Harrow, England
+2;1;Wright, Miss. Marion;female;26;0;0;220844;13,5;;S;9;;Yoevil, England / Cottage Grove, OR
+2;0;"Yrois, Miss. Henriette (""Mrs Harbeck"")";female;24;0;0;248747;13;;S;;;Paris
+3;0;Abbing, Mr. Anthony;male;42;0;0;C.A. 5547;7,55;;S;;;
+3;0;Abbott, Master. Eugene Joseph;male;13;0;2;C.A. 2673;20,25;;S;;;East Providence, RI
+3;0;Abbott, Mr. Rossmore Edward;male;16;1;1;C.A. 2673;20,25;;S;;190;East Providence, RI
+3;1;Abbott, Mrs. Stanton (Rosa Hunt);female;35;1;1;C.A. 2673;20,25;;S;A;;East Providence, RI
+3;1;Abelseth, Miss. Karen Marie;female;16;0;0;348125;7,65;;S;16;;Norway Los Angeles, CA
+3;1;Abelseth, Mr. Olaus Jorgensen;male;25;0;0;348122;7,65;F G63;S;A;;Perkins County, SD
+3;1;Abrahamsson, Mr. Abraham August Johannes;male;20;0;0;SOTON/O2 3101284;7,925;;S;15;;Taalintehdas, Finland Hoboken, NJ
+3;1;Abrahim, Mrs. Joseph (Sophie Halaut Easu);female;18;0;0;2657;7,2292;;C;C;;Greensburg, PA
+3;0;Adahl, Mr. Mauritz Nils Martin;male;30;0;0;C 7076;7,25;;S;;72;Asarum, Sweden Brooklyn, NY
+3;0;Adams, Mr. John;male;26;0;0;341826;8,05;;S;;103;Bournemouth, England
+3;0;Ahlin, Mrs. Johan (Johanna Persdotter Larsson);female;40;1;0;7546;9,475;;S;;;Sweden Akeley, MN
+3;1;Aks, Master. Philip Frank;male;0,8333;0;1;392091;9,35;;S;11;;London, England Norfolk, VA
+3;1;Aks, Mrs. Sam (Leah Rosen);female;18;0;1;392091;9,35;;S;13;;London, England Norfolk, VA
+3;1;Albimona, Mr. Nassef Cassem;male;26;0;0;2699;18,7875;;C;15;;Syria Fredericksburg, VA
+3;0;Alexander, Mr. William;male;26;0;0;3474;7,8875;;S;;;England Albion, NY
+3;0;Alhomaki, Mr. Ilmari Rudolf;male;20;0;0;SOTON/O2 3101287;7,925;;S;;;Salo, Finland Astoria, OR
+3;0;Ali, Mr. Ahmed;male;24;0;0;SOTON/O.Q. 3101311;7,05;;S;;;
+3;0;Ali, Mr. William;male;25;0;0;SOTON/O.Q. 3101312;7,05;;S;;79;Argentina
+3;0;Allen, Mr. William Henry;male;35;0;0;373450;8,05;;S;;;Lower Clapton, Middlesex or Erdington, Birmingham
+3;0;Allum, Mr. Owen George;male;18;0;0;2223;8,3;;S;;259;Windsor, England New York, NY
+3;0;Andersen, Mr. Albert Karvin;male;32;0;0;C 4001;22,525;;S;;260;Bergen, Norway
+3;1;Andersen-Jensen, Miss. Carla Christine Nielsine;female;19;1;0;350046;7,8542;;S;16;;
+3;0;Andersson, Master. Sigvard Harald Elias;male;4;4;2;347082;31,275;;S;;;Sweden Winnipeg, MN
+3;0;Andersson, Miss. Ebba Iris Alfrida;female;6;4;2;347082;31,275;;S;;;Sweden Winnipeg, MN
+3;0;Andersson, Miss. Ellis Anna Maria;female;2;4;2;347082;31,275;;S;;;Sweden Winnipeg, MN
+3;1;Andersson, Miss. Erna Alexandra;female;17;4;2;3101281;7,925;;S;D;;Ruotsinphyhtaa, Finland New York, NY
+3;0;Andersson, Miss. Ida Augusta Margareta;female;38;4;2;347091;7,775;;S;;;Vadsbro, Sweden Ministee, MI
+3;0;Andersson, Miss. Ingeborg Constanzia;female;9;4;2;347082;31,275;;S;;;Sweden Winnipeg, MN
+3;0;Andersson, Miss. Sigrid Elisabeth;female;11;4;2;347082;31,275;;S;;;Sweden Winnipeg, MN
+3;0;Andersson, Mr. Anders Johan;male;39;1;5;347082;31,275;;S;;;Sweden Winnipeg, MN
+3;1;"Andersson, Mr. August Edvard (""Wennerstrom"")";male;27;0;0;350043;7,7958;;S;A;;
+3;0;Andersson, Mr. Johan Samuel;male;26;0;0;347075;7,775;;S;;;Hartford, CT
+3;0;Andersson, Mrs. Anders Johan (Alfrida Konstantia Brogren);female;39;1;5;347082;31,275;;S;;;Sweden Winnipeg, MN
+3;0;Andreasson, Mr. Paul Edvin;male;20;0;0;347466;7,8542;;S;;;Sweden Chicago, IL
+3;0;Angheloff, Mr. Minko;male;26;0;0;349202;7,8958;;S;;;Bulgaria Chicago, IL
+3;0;Arnold-Franchi, Mr. Josef;male;25;1;0;349237;17,8;;S;;;Altdorf, Switzerland
+3;0;Arnold-Franchi, Mrs. Josef (Josefine Franchi);female;18;1;0;349237;17,8;;S;;;Altdorf, Switzerland
+3;0;Aronsson, Mr. Ernst Axel Algot;male;24;0;0;349911;7,775;;S;;;Sweden Joliet, IL
+3;0;Asim, Mr. Adola;male;35;0;0;SOTON/O.Q. 3101310;7,05;;S;;;
+3;0;Asplund, Master. Carl Edgar;male;5;4;2;347077;31,3875;;S;;;Sweden  Worcester, MA
+3;0;Asplund, Master. Clarence Gustaf Hugo;male;9;4;2;347077;31,3875;;S;;;Sweden Worcester, MA
+3;1;Asplund, Master. Edvin Rojj Felix;male;3;4;2;347077;31,3875;;S;15;;Sweden Worcester, MA
+3;0;Asplund, Master. Filip Oscar;male;13;4;2;347077;31,3875;;S;;;Sweden Worcester, MA
+3;1;Asplund, Miss. Lillian Gertrud;female;5;4;2;347077;31,3875;;S;15;;Sweden Worcester, MA
+3;0;Asplund, Mr. Carl Oscar Vilhelm Gustafsson;male;40;1;5;347077;31,3875;;S;;142;Sweden  Worcester, MA
+3;1;Asplund, Mr. Johan Charles;male;23;0;0;350054;7,7958;;S;13;;Oskarshamn, Sweden Minneapolis, MN
+3;1;Asplund, Mrs. Carl Oscar (Selma Augusta Emilia Johansson);female;38;1;5;347077;31,3875;;S;15;;Sweden  Worcester, MA
+3;1;"Assaf Khalil, Mrs. Mariana (""Miriam"")";female;45;0;0;2696;7,225;;C;C;;Ottawa, ON
+3;0;Assaf, Mr. Gerios;male;21;0;0;2692;7,225;;C;;;Ottawa, ON
+3;0;Assam, Mr. Ali;male;23;0;0;SOTON/O.Q. 3101309;7,05;;S;;;
+3;0;Attalah, Miss. Malake;female;17;0;0;2627;14,4583;;C;;;
+3;0;Attalah, Mr. Sleiman;male;30;0;0;2694;7,225;;C;;;Ottawa, ON
+3;0;Augustsson, Mr. Albert;male;23;0;0;347468;7,8542;;S;;;Krakoryd, Sweden Bloomington, IL
+3;1;Ayoub, Miss. Banoura;female;13;0;0;2687;7,2292;;C;C;;Syria Youngstown, OH
+3;0;Baccos, Mr. Raffull;male;20;0;0;2679;7,225;;C;;;
+3;0;Backstrom, Mr. Karl Alfred;male;32;1;0;3101278;15,85;;S;D;;Ruotsinphytaa, Finland New York, NY
+3;1;Backstrom, Mrs. Karl Alfred (Maria Mathilda Gustafsson);female;33;3;0;3101278;15,85;;S;;;Ruotsinphytaa, Finland New York, NY
+3;1;Baclini, Miss. Eugenie;female;0,75;2;1;2666;19,2583;;C;C;;Syria New York, NY
+3;1;Baclini, Miss. Helene Barbara;female;0,75;2;1;2666;19,2583;;C;C;;Syria New York, NY
+3;1;Baclini, Miss. Marie Catherine;female;5;2;1;2666;19,2583;;C;C;;Syria New York, NY
+3;1;Baclini, Mrs. Solomon (Latifa Qurban);female;24;0;3;2666;19,2583;;C;C;;Syria New York, NY
+3;1;Badman, Miss. Emily Louisa;female;18;0;0;A/4 31416;8,05;;S;C;;London Skanteales, NY
+3;0;Badt, Mr. Mohamed;male;40;0;0;2623;7,225;;C;;;
+3;0;Balkic, Mr. Cerin;male;26;0;0;349248;7,8958;;S;;;
+3;1;Barah, Mr. Hanna Assi;male;20;0;0;2663;7,2292;;C;15;;
+3;0;Barbara, Miss. Saiide;female;18;0;1;2691;14,4542;;C;;;Syria Ottawa, ON
+3;0;Barbara, Mrs. (Catherine David);female;45;0;1;2691;14,4542;;C;;;Syria Ottawa, ON
+3;0;Barry, Miss. Julia;female;27;0;0;330844;7,8792;;Q;;;New York, NY
+3;0;Barton, Mr. David John;male;22;0;0;324669;8,05;;S;;;England New York, NY
+3;0;Beavan, Mr. William Thomas;male;19;0;0;323951;8,05;;S;;;England
+3;0;Bengtsson, Mr. John Viktor;male;26;0;0;347068;7,775;;S;;;Krakudden, Sweden Moune, IL
+3;0;Berglund, Mr. Karl Ivar Sven;male;22;0;0;PP 4348;9,35;;S;;;Tranvik, Finland New York
+3;0;Betros, Master. Seman;male;;0;0;2622;7,2292;;C;;;
+3;0;Betros, Mr. Tannous;male;20;0;0;2648;4,0125;;C;;;Syria
+3;1;Bing, Mr. Lee;male;32;0;0;1601;56,4958;;S;C;;Hong Kong New York, NY
+3;0;Birkeland, Mr. Hans Martin Monsen;male;21;0;0;312992;7,775;;S;;;Brennes, Norway New York
+3;0;Bjorklund, Mr. Ernst Herbert;male;18;0;0;347090;7,75;;S;;;Stockholm, Sweden New York
+3;0;Bostandyeff, Mr. Guentcho;male;26;0;0;349224;7,8958;;S;;;Bulgaria Chicago, IL
+3;0;Boulos, Master. Akar;male;6;1;1;2678;15,2458;;C;;;Syria Kent, ON
+3;0;Boulos, Miss. Nourelain;female;9;1;1;2678;15,2458;;C;;;Syria Kent, ON
+3;0;Boulos, Mr. Hanna;male;;0;0;2664;7,225;;C;;;Syria
+3;0;Boulos, Mrs. Joseph (Sultana);female;;0;2;2678;15,2458;;C;;;Syria Kent, ON
+3;0;Bourke, Miss. Mary;female;;0;2;364848;7,75;;Q;;;Ireland Chicago, IL
+3;0;Bourke, Mr. John;male;40;1;1;364849;15,5;;Q;;;Ireland Chicago, IL
+3;0;Bourke, Mrs. John (Catherine);female;32;1;1;364849;15,5;;Q;;;Ireland Chicago, IL
+3;0;"Bowen, Mr. David John ""Dai""";male;21;0;0;54636;16,1;;S;;;Treherbert, Cardiff, Wales
+3;1;Bradley, Miss. Bridget Delia;female;22;0;0;334914;7,725;;Q;13;;Kingwilliamstown, Co Cork, Ireland Glens Falls, NY
+3;0;Braf, Miss. Elin Ester Maria;female;20;0;0;347471;7,8542;;S;;;Medeltorp, Sweden Chicago, IL
+3;0;Braund, Mr. Lewis Richard;male;29;1;0;3460;7,0458;;S;;;Bridgerule, Devon
+3;0;Braund, Mr. Owen Harris;male;22;1;0;A/5 21171;7,25;;S;;;Bridgerule, Devon
+3;0;Brobeck, Mr. Karl Rudolf;male;22;0;0;350045;7,7958;;S;;;Sweden Worcester, MA
+3;0;Brocklebank, Mr. William Alfred;male;35;0;0;364512;8,05;;S;;;Broomfield, Chelmsford, England
+3;0;Buckley, Miss. Katherine;female;18,5;0;0;329944;7,2833;;Q;;299;Co Cork, Ireland Roxbury, MA
+3;1;Buckley, Mr. Daniel;male;21;0;0;330920;7,8208;;Q;13;;Kingwilliamstown, Co Cork, Ireland New York, NY
+3;0;Burke, Mr. Jeremiah;male;19;0;0;365222;6,75;;Q;;;Co Cork, Ireland Charlestown, MA
+3;0;Burns, Miss. Mary Delia;female;18;0;0;330963;7,8792;;Q;;;Co Sligo, Ireland New York, NY
+3;0;Cacic, Miss. Manda;female;21;0;0;315087;8,6625;;S;;;
+3;0;Cacic, Miss. Marija;female;30;0;0;315084;8,6625;;S;;;
+3;0;Cacic, Mr. Jego Grga;male;18;0;0;315091;8,6625;;S;;;
+3;0;Cacic, Mr. Luka;male;38;0;0;315089;8,6625;;S;;;Croatia
+3;0;Calic, Mr. Jovo;male;17;0;0;315093;8,6625;;S;;;
+3;0;Calic, Mr. Petar;male;17;0;0;315086;8,6625;;S;;;
+3;0;Canavan, Miss. Mary;female;21;0;0;364846;7,75;;Q;;;
+3;0;Canavan, Mr. Patrick;male;21;0;0;364858;7,75;;Q;;;Ireland Philadelphia, PA
+3;0;Cann, Mr. Ernest Charles;male;21;0;0;A./5. 2152;8,05;;S;;;
+3;0;Caram, Mr. Joseph;male;;1;0;2689;14,4583;;C;;;Ottawa, ON
+3;0;Caram, Mrs. Joseph (Maria Elias);female;;1;0;2689;14,4583;;C;;;Ottawa, ON
+3;0;Carlsson, Mr. August Sigfrid;male;28;0;0;350042;7,7958;;S;;;Dagsas, Sweden Fower, MN
+3;0;Carlsson, Mr. Carl Robert;male;24;0;0;350409;7,8542;;S;;;Goteborg, Sweden Huntley, IL
+3;1;"Carr, Miss. Helen ""Ellen""";female;16;0;0;367231;7,75;;Q;16;;Co Longford, Ireland New York, NY
+3;0;Carr, Miss. Jeannie;female;37;0;0;368364;7,75;;Q;;;Co Sligo, Ireland Hartford, CT
+3;0;Carver, Mr. Alfred John;male;28;0;0;392095;7,25;;S;;;St Denys, Southampton, Hants
+3;0;Celotti, Mr. Francesco;male;24;0;0;343275;8,05;;S;;;London
+3;0;Charters, Mr. David;male;21;0;0;A/5. 13032;7,7333;;Q;;;Ireland New York, NY
+3;1;Chip, Mr. Chang;male;32;0;0;1601;56,4958;;S;C;;Hong Kong New York, NY
+3;0;Christmann, Mr. Emil;male;29;0;0;343276;8,05;;S;;;
+3;0;Chronopoulos, Mr. Apostolos;male;26;1;0;2680;14,4542;;C;;;Greece
+3;0;Chronopoulos, Mr. Demetrios;male;18;1;0;2680;14,4542;;C;;;Greece
+3;0;Coelho, Mr. Domingos Fernandeo;male;20;0;0;SOTON/O.Q. 3101307;7,05;;S;;;Portugal
+3;1;"Cohen, Mr. Gurshon ""Gus""";male;18;0;0;A/5 3540;8,05;;S;12;;London Brooklyn, NY
+3;0;Colbert, Mr. Patrick;male;24;0;0;371109;7,25;;Q;;;Co Limerick, Ireland Sherbrooke, PQ
+3;0;Coleff, Mr. Peju;male;36;0;0;349210;7,4958;;S;;;Bulgaria Chicago, IL
+3;0;Coleff, Mr. Satio;male;24;0;0;349209;7,4958;;S;;;
+3;0;Conlon, Mr. Thomas Henry;male;31;0;0;21332;7,7333;;Q;;;Philadelphia, PA
+3;0;Connaghton, Mr. Michael;male;31;0;0;335097;7,75;;Q;;;Ireland Brooklyn, NY
+3;1;Connolly, Miss. Kate;female;22;0;0;370373;7,75;;Q;13;;Ireland
+3;0;Connolly, Miss. Kate;female;30;0;0;330972;7,6292;;Q;;;Ireland
+3;0;Connors, Mr. Patrick;male;70,5;0;0;370369;7,75;;Q;;171;
+3;0;Cook, Mr. Jacob;male;43;0;0;A/5 3536;8,05;;S;;;
+3;0;Cor, Mr. Bartol;male;35;0;0;349230;7,8958;;S;;;Austria
+3;0;Cor, Mr. Ivan;male;27;0;0;349229;7,8958;;S;;;Austria
+3;0;Cor, Mr. Liudevit;male;19;0;0;349231;7,8958;;S;;;Austria
+3;0;Corn, Mr. Harry;male;30;0;0;SOTON/OQ 392090;8,05;;S;;;London
+3;1;"Coutts, Master. Eden Leslie ""Neville""";male;9;1;1;C.A. 37671;15,9;;S;2;;England Brooklyn, NY
+3;1;"Coutts, Master. William Loch ""William""";male;3;1;1;C.A. 37671;15,9;;S;2;;England Brooklyn, NY
+3;1;"Coutts, Mrs. William (Winnie ""Minnie"" Treanor)";female;36;0;2;C.A. 37671;15,9;;S;2;;England Brooklyn, NY
+3;0;Coxon, Mr. Daniel;male;59;0;0;364500;7,25;;S;;;Merrill, WI
+3;0;Crease, Mr. Ernest James;male;19;0;0;S.P. 3464;8,1583;;S;;;Bristol, England Cleveland, OH
+3;1;Cribb, Miss. Laura Alice;female;17;0;1;371362;16,1;;S;12;;Bournemouth, England Newark, NJ
+3;0;Cribb, Mr. John Hatfield;male;44;0;1;371362;16,1;;S;;;Bournemouth, England Newark, NJ
+3;0;Culumovic, Mr. Jeso;male;17;0;0;315090;8,6625;;S;;;Austria-Hungary
+3;0;Daher, Mr. Shedid;male;22,5;0;0;2698;7,225;;C;;9;
+3;1;Dahl, Mr. Karl Edwart;male;45;0;0;7598;8,05;;S;15;;Australia Fingal, ND
+3;0;Dahlberg, Miss. Gerda Ulrika;female;22;0;0;7552;10,5167;;S;;;Norrlot, Sweden Chicago, IL
+3;0;Dakic, Mr. Branko;male;19;0;0;349228;10,1708;;S;;;Austria
+3;1;"Daly, Miss. Margaret Marcella ""Maggie""";female;30;0;0;382650;6,95;;Q;15;;Co Athlone, Ireland New York, NY
+3;1;Daly, Mr. Eugene Patrick;male;29;0;0;382651;7,75;;Q;13 15 B;;Co Athlone, Ireland New York, NY
+3;0;Danbom, Master. Gilbert Sigvard Emanuel;male;0,3333;0;2;347080;14,4;;S;;;Stanton, IA
+3;0;Danbom, Mr. Ernst Gilbert;male;34;1;1;347080;14,4;;S;;197;Stanton, IA
+3;0;Danbom, Mrs. Ernst Gilbert (Anna Sigrid Maria Brogren);female;28;1;1;347080;14,4;;S;;;Stanton, IA
+3;0;Danoff, Mr. Yoto;male;27;0;0;349219;7,8958;;S;;;Bulgaria Chicago, IL
+3;0;Dantcheff, Mr. Ristiu;male;25;0;0;349203;7,8958;;S;;;Bulgaria Chicago, IL
+3;0;Davies, Mr. Alfred J;male;24;2;0;A/4 48871;24,15;;S;;;West Bromwich, England Pontiac, MI
+3;0;Davies, Mr. Evan;male;22;0;0;SC/A4 23568;8,05;;S;;;
+3;0;Davies, Mr. John Samuel;male;21;2;0;A/4 48871;24,15;;S;;;West Bromwich, England Pontiac, MI
+3;0;Davies, Mr. Joseph;male;17;2;0;A/4 48873;8,05;;S;;;West Bromwich, England Pontiac, MI
+3;0;Davison, Mr. Thomas Henry;male;;1;0;386525;16,1;;S;;;Liverpool, England Bedford, OH
+3;1;Davison, Mrs. Thomas Henry (Mary E Finck);female;;1;0;386525;16,1;;S;16;;Liverpool, England Bedford, OH
+3;1;de Messemaeker, Mr. Guillaume Joseph;male;36,5;1;0;345572;17,4;;S;15;;Tampico, MT
+3;1;de Messemaeker, Mrs. Guillaume Joseph (Emma);female;36;1;0;345572;17,4;;S;13;;Tampico, MT
+3;1;de Mulder, Mr. Theodore;male;30;0;0;345774;9,5;;S;11;;Belgium Detroit, MI
+3;0;de Pelsmaeker, Mr. Alfons;male;16;0;0;345778;9,5;;S;;;
+3;1;Dean, Master. Bertram Vere;male;1;1;2;C.A. 2315;20,575;;S;10;;Devon, England Wichita, KS
+3;1;"Dean, Miss. Elizabeth Gladys ""Millvina""";female;0,1667;1;2;C.A. 2315;20,575;;S;10;;Devon, England Wichita, KS
+3;0;Dean, Mr. Bertram Frank;male;26;1;2;C.A. 2315;20,575;;S;;;Devon, England Wichita, KS
+3;1;Dean, Mrs. Bertram (Eva Georgetta Light);female;33;1;2;C.A. 2315;20,575;;S;10;;Devon, England Wichita, KS
+3;0;Delalic, Mr. Redjo;male;25;0;0;349250;7,8958;;S;;;
+3;0;Demetri, Mr. Marinko;male;;0;0;349238;7,8958;;S;;;
+3;0;Denkoff, Mr. Mitto;male;;0;0;349225;7,8958;;S;;;Bulgaria Coon Rapids, IA
+3;0;Dennis, Mr. Samuel;male;22;0;0;A/5 21172;7,25;;S;;;
+3;0;Dennis, Mr. William;male;36;0;0;A/5 21175;7,25;;S;;;
+3;1;Devaney, Miss. Margaret Delia;female;19;0;0;330958;7,8792;;Q;C;;Kilmacowen, Co Sligo, Ireland New York, NY
+3;0;Dika, Mr. Mirko;male;17;0;0;349232;7,8958;;S;;;
+3;0;Dimic, Mr. Jovan;male;42;0;0;315088;8,6625;;S;;;
+3;0;Dintcheff, Mr. Valtcho;male;43;0;0;349226;7,8958;;S;;;
+3;0;Doharr, Mr. Tannous;male;;0;0;2686;7,2292;;C;;;
+3;0;Dooley, Mr. Patrick;male;32;0;0;370376;7,75;;Q;;;Ireland New York, NY
+3;1;Dorking, Mr. Edward Arthur;male;19;0;0;A/5. 10482;8,05;;S;B;;England Oglesby, IL
+3;1;Dowdell, Miss. Elizabeth;female;30;0;0;364516;12,475;;S;13;;Union Hill, NJ
+3;0;Doyle, Miss. Elizabeth;female;24;0;0;368702;7,75;;Q;;;Ireland New York, NY
+3;1;Drapkin, Miss. Jennie;female;23;0;0;SOTON/OQ 392083;8,05;;S;;;London New York, NY
+3;0;Drazenoic, Mr. Jozef;male;33;0;0;349241;7,8958;;C;;51;Austria Niagara Falls, NY
+3;0;Duane, Mr. Frank;male;65;0;0;336439;7,75;;Q;;;
+3;1;Duquemin, Mr. Joseph;male;24;0;0;S.O./P.P. 752;7,55;;S;D;;England Albion, NY
+3;0;Dyker, Mr. Adolf Fredrik;male;23;1;0;347072;13,9;;S;;;West Haven, CT
+3;1;Dyker, Mrs. Adolf Fredrik (Anna Elisabeth Judith Andersson);female;22;1;0;347072;13,9;;S;16;;West Haven, CT
+3;0;Edvardsson, Mr. Gustaf Hjalmar;male;18;0;0;349912;7,775;;S;;;Tofta, Sweden Joliet, IL
+3;0;Eklund, Mr. Hans Linus;male;16;0;0;347074;7,775;;S;;;Karberg, Sweden Jerome Junction, AZ
+3;0;Ekstrom, Mr. Johan;male;45;0;0;347061;6,975;;S;;;Effington Rut, SD
+3;0;Elias, Mr. Dibo;male;;0;0;2674;7,225;;C;;;
+3;0;Elias, Mr. Joseph;male;39;0;2;2675;7,2292;;C;;;Syria Ottawa, ON
+3;0;Elias, Mr. Joseph Jr;male;17;1;1;2690;7,2292;;C;;;
+3;0;Elias, Mr. Tannous;male;15;1;1;2695;7,2292;;C;;;Syria
+3;0;Elsbury, Mr. William James;male;47;0;0;A/5 3902;7,25;;S;;;Illinois, USA
+3;1;Emanuel, Miss. Virginia Ethel;female;5;0;0;364516;12,475;;S;13;;New York, NY
+3;0;Emir, Mr. Farred Chehab;male;;0;0;2631;7,225;;C;;;
+3;0;Everett, Mr. Thomas James;male;40,5;0;0;C.A. 6212;15,1;;S;;187;
+3;0;Farrell, Mr. James;male;40,5;0;0;367232;7,75;;Q;;68;Aughnacliff, Co Longford, Ireland New York, NY
+3;1;Finoli, Mr. Luigi;male;;0;0;SOTON/O.Q. 3101308;7,05;;S;15;;Italy Philadelphia, PA
+3;0;Fischer, Mr. Eberhard Thelander;male;18;0;0;350036;7,7958;;S;;;
+3;0;Fleming, Miss. Honora;female;;0;0;364859;7,75;;Q;;;
+3;0;Flynn, Mr. James;male;;0;0;364851;7,75;;Q;;;
+3;0;Flynn, Mr. John;male;;0;0;368323;6,95;;Q;;;
+3;0;Foley, Mr. Joseph;male;26;0;0;330910;7,8792;;Q;;;Ireland Chicago, IL
+3;0;Foley, Mr. William;male;;0;0;365235;7,75;;Q;;;Ireland
+3;1;Foo, Mr. Choong;male;;0;0;1601;56,4958;;S;13;;Hong Kong New York, NY
+3;0;"Ford, Miss. Doolina Margaret ""Daisy""";female;21;2;2;W./C. 6608;34,375;;S;;;Rotherfield, Sussex, England Essex Co, MA
+3;0;"Ford, Miss. Robina Maggie ""Ruby""";female;9;2;2;W./C. 6608;34,375;;S;;;Rotherfield, Sussex, England Essex Co, MA
+3;0;Ford, Mr. Arthur;male;;0;0;A/5 1478;8,05;;S;;;Bridgwater, Somerset, England
+3;0;Ford, Mr. Edward Watson;male;18;2;2;W./C. 6608;34,375;;S;;;Rotherfield, Sussex, England Essex Co, MA
+3;0;Ford, Mr. William Neal;male;16;1;3;W./C. 6608;34,375;;S;;;Rotherfield, Sussex, England Essex Co, MA
+3;0;Ford, Mrs. Edward (Margaret Ann Watson);female;48;1;3;W./C. 6608;34,375;;S;;;Rotherfield, Sussex, England Essex Co, MA
+3;0;Fox, Mr. Patrick;male;;0;0;368573;7,75;;Q;;;Ireland New York, NY
+3;0;Franklin, Mr. Charles (Charles Fardon);male;;0;0;SOTON/O.Q. 3101314;7,25;;S;;;
+3;0;Gallagher, Mr. Martin;male;25;0;0;36864;7,7417;;Q;;;New York, NY
+3;0;Garfirth, Mr. John;male;;0;0;358585;14,5;;S;;;
+3;0;Gheorgheff, Mr. Stanio;male;;0;0;349254;7,8958;;C;;;
+3;0;Gilinski, Mr. Eliezer;male;22;0;0;14973;8,05;;S;;47;
+3;1;"Gilnagh, Miss. Katherine ""Katie""";female;16;0;0;35851;7,7333;;Q;16;;Co Longford, Ireland New York, NY
+3;1;Glynn, Miss. Mary Agatha;female;;0;0;335677;7,75;;Q;13;;Co Clare, Ireland Washington, DC
+3;1;"Goldsmith, Master. Frank John William ""Frankie""";male;9;0;2;363291;20,525;;S;C D;;Strood, Kent, England Detroit, MI
+3;0;Goldsmith, Mr. Frank John;male;33;1;1;363291;20,525;;S;;;Strood, Kent, England Detroit, MI
+3;0;Goldsmith, Mr. Nathan;male;41;0;0;SOTON/O.Q. 3101263;7,85;;S;;;Philadelphia, PA
+3;1;Goldsmith, Mrs. Frank John (Emily Alice Brown);female;31;1;1;363291;20,525;;S;C D;;Strood, Kent, England Detroit, MI
+3;0;Goncalves, Mr. Manuel Estanslas;male;38;0;0;SOTON/O.Q. 3101306;7,05;;S;;;Portugal
+3;0;Goodwin, Master. Harold Victor;male;9;5;2;CA 2144;46,9;;S;;;Wiltshire, England Niagara Falls, NY
+3;0;Goodwin, Master. Sidney Leonard;male;1;5;2;CA 2144;46,9;;S;;;Wiltshire, England Niagara Falls, NY
+3;0;Goodwin, Master. William Frederick;male;11;5;2;CA 2144;46,9;;S;;;Wiltshire, England Niagara Falls, NY
+3;0;Goodwin, Miss. Jessie Allis;female;10;5;2;CA 2144;46,9;;S;;;Wiltshire, England Niagara Falls, NY
+3;0;Goodwin, Miss. Lillian Amy;female;16;5;2;CA 2144;46,9;;S;;;Wiltshire, England Niagara Falls, NY
+3;0;Goodwin, Mr. Charles Edward;male;14;5;2;CA 2144;46,9;;S;;;Wiltshire, England Niagara Falls, NY
+3;0;Goodwin, Mr. Charles Frederick;male;40;1;6;CA 2144;46,9;;S;;;Wiltshire, England Niagara Falls, NY
+3;0;Goodwin, Mrs. Frederick (Augusta Tyler);female;43;1;6;CA 2144;46,9;;S;;;Wiltshire, England Niagara Falls, NY
+3;0;Green, Mr. George Henry;male;51;0;0;21440;8,05;;S;;;Dorking, Surrey, England
+3;0;Gronnestad, Mr. Daniel Danielsen;male;32;0;0;8471;8,3625;;S;;;Foresvik, Norway Portland, ND
+3;0;Guest, Mr. Robert;male;;0;0;376563;8,05;;S;;;
+3;0;Gustafsson, Mr. Alfred Ossian;male;20;0;0;7534;9,8458;;S;;;Waukegan, Chicago, IL
+3;0;Gustafsson, Mr. Anders Vilhelm;male;37;2;0;3101276;7,925;;S;;98;Ruotsinphytaa, Finland New York, NY
+3;0;Gustafsson, Mr. Johan Birger;male;28;2;0;3101277;7,925;;S;;;Ruotsinphytaa, Finland New York, NY
+3;0;Gustafsson, Mr. Karl Gideon;male;19;0;0;347069;7,775;;S;;;Myren, Sweden New York, NY
+3;0;Haas, Miss. Aloisia;female;24;0;0;349236;8,85;;S;;;
+3;0;Hagardon, Miss. Kate;female;17;0;0;AQ/3. 30631;7,7333;;Q;;;
+3;0;Hagland, Mr. Ingvald Olai Olsen;male;;1;0;65303;19,9667;;S;;;
+3;0;Hagland, Mr. Konrad Mathias Reiersen;male;;1;0;65304;19,9667;;S;;;
+3;0;Hakkarainen, Mr. Pekka Pietari;male;28;1;0;STON/O2. 3101279;15,85;;S;;;
+3;1;Hakkarainen, Mrs. Pekka Pietari (Elin Matilda Dolck);female;24;1;0;STON/O2. 3101279;15,85;;S;15;;
+3;0;Hampe, Mr. Leon;male;20;0;0;345769;9,5;;S;;;
+3;0;Hanna, Mr. Mansour;male;23,5;0;0;2693;7,2292;;C;;188;
+3;0;Hansen, Mr. Claus Peter;male;41;2;0;350026;14,1083;;S;;;
+3;0;Hansen, Mr. Henrik Juul;male;26;1;0;350025;7,8542;;S;;;
+3;0;Hansen, Mr. Henry Damsgaard;male;21;0;0;350029;7,8542;;S;;69;
+3;1;Hansen, Mrs. Claus Peter (Jennie L Howard);female;45;1;0;350026;14,1083;;S;11;;
+3;0;Harknett, Miss. Alice Phoebe;female;;0;0;W./C. 6609;7,55;;S;;;
+3;0;Harmer, Mr. Abraham (David Lishin);male;25;0;0;374887;7,25;;S;B;;
+3;0;Hart, Mr. Henry;male;;0;0;394140;6,8583;;Q;;;
+3;0;Hassan, Mr. Houssein G N;male;11;0;0;2699;18,7875;;C;;;
+3;1;"Healy, Miss. Hanora ""Nora""";female;;0;0;370375;7,75;;Q;16;;
+3;1;Hedman, Mr. Oskar Arvid;male;27;0;0;347089;6,975;;S;15;;
+3;1;Hee, Mr. Ling;male;;0;0;1601;56,4958;;S;C;;
+3;0;"Hegarty, Miss. Hanora ""Nora""";female;18;0;0;365226;6,75;;Q;;;
+3;1;Heikkinen, Miss. Laina;female;26;0;0;STON/O2. 3101282;7,925;;S;;;
+3;0;Heininen, Miss. Wendla Maria;female;23;0;0;STON/O2. 3101290;7,925;;S;;;
+3;1;Hellstrom, Miss. Hilda Maria;female;22;0;0;7548;8,9625;;S;C;;
+3;0;Hendekovic, Mr. Ignjac;male;28;0;0;349243;7,8958;;S;;306;
+3;0;Henriksson, Miss. Jenny Lovisa;female;28;0;0;347086;7,775;;S;;;
+3;0;Henry, Miss. Delia;female;;0;0;382649;7,75;;Q;;;
+3;1;Hirvonen, Miss. Hildur E;female;2;0;1;3101298;12,2875;;S;15;;
+3;1;Hirvonen, Mrs. Alexander (Helga E Lindqvist);female;22;1;1;3101298;12,2875;;S;15;;
+3;0;Holm, Mr. John Fredrik Alexander;male;43;0;0;C 7075;6,45;;S;;;
+3;0;Holthen, Mr. Johan Martin;male;28;0;0;C 4001;22,525;;S;;;
+3;1;Honkanen, Miss. Eliina;female;27;0;0;STON/O2. 3101283;7,925;;S;;;
+3;0;Horgan, Mr. John;male;;0;0;370377;7,75;;Q;;;
+3;1;Howard, Miss. May Elizabeth;female;;0;0;A. 2. 39186;8,05;;S;C;;
+3;0;Humblen, Mr. Adolf Mathias Nicolai Olsen;male;42;0;0;348121;7,65;F G63;S;;120;
+3;1;Hyman, Mr. Abraham;male;;0;0;3470;7,8875;;S;C;;
+3;0;Ibrahim Shawah, Mr. Yousseff;male;30;0;0;2685;7,2292;;C;;;
+3;0;Ilieff, Mr. Ylio;male;;0;0;349220;7,8958;;S;;;
+3;0;Ilmakangas, Miss. Ida Livija;female;27;1;0;STON/O2. 3101270;7,925;;S;;;
+3;0;Ilmakangas, Miss. Pieta Sofia;female;25;1;0;STON/O2. 3101271;7,925;;S;;;
+3;0;Ivanoff, Mr. Kanio;male;;0;0;349201;7,8958;;S;;;
+3;1;Jalsevac, Mr. Ivan;male;29;0;0;349240;7,8958;;C;15;;
+3;1;Jansson, Mr. Carl Olof;male;21;0;0;350034;7,7958;;S;A;;
+3;0;Jardin, Mr. Jose Neto;male;;0;0;SOTON/O.Q. 3101305;7,05;;S;;;
+3;0;Jensen, Mr. Hans Peder;male;20;0;0;350050;7,8542;;S;;;
+3;0;Jensen, Mr. Niels Peder;male;48;0;0;350047;7,8542;;S;;;
+3;0;Jensen, Mr. Svend Lauritz;male;17;1;0;350048;7,0542;;S;;;
+3;1;Jermyn, Miss. Annie;female;;0;0;14313;7,75;;Q;D;;
+3;1;Johannesen-Bratthammer, Mr. Bernt;male;;0;0;65306;8,1125;;S;13;;
+3;0;Johanson, Mr. Jakob Alfred;male;34;0;0;3101264;6,4958;;S;;143;
+3;1;Johansson Palmquist, Mr. Oskar Leander;male;26;0;0;347070;7,775;;S;15;;
+3;0;Johansson, Mr. Erik;male;22;0;0;350052;7,7958;;S;;156;
+3;0;Johansson, Mr. Gustaf Joel;male;33;0;0;7540;8,6542;;S;;285;
+3;0;Johansson, Mr. Karl Johan;male;31;0;0;347063;7,775;;S;;;
+3;0;Johansson, Mr. Nils;male;29;0;0;347467;7,8542;;S;;;
+3;1;Johnson, Master. Harold Theodor;male;4;1;1;347742;11,1333;;S;15;;
+3;1;Johnson, Miss. Eleanor Ileen;female;1;1;1;347742;11,1333;;S;15;;
+3;0;Johnson, Mr. Alfred;male;49;0;0;LINE;0;;S;;;
+3;0;Johnson, Mr. Malkolm Joackim;male;33;0;0;347062;7,775;;S;;37;
+3;0;Johnson, Mr. William Cahoone Jr;male;19;0;0;LINE;0;;S;;;
+3;1;Johnson, Mrs. Oscar W (Elisabeth Vilhelmina Berg);female;27;0;2;347742;11,1333;;S;15;;
+3;0;"Johnston, Master. William Arthur ""Willie""";male;;1;2;W./C. 6607;23,45;;S;;;
+3;0;"Johnston, Miss. Catherine Helen ""Carrie""";female;;1;2;W./C. 6607;23,45;;S;;;
+3;0;Johnston, Mr. Andrew G;male;;1;2;W./C. 6607;23,45;;S;;;
+3;0;"Johnston, Mrs. Andrew G (Elizabeth ""Lily"" Watson)";female;;1;2;W./C. 6607;23,45;;S;;;
+3;0;Jonkoff, Mr. Lalio;male;23;0;0;349204;7,8958;;S;;;
+3;1;Jonsson, Mr. Carl;male;32;0;0;350417;7,8542;;S;15;;
+3;0;Jonsson, Mr. Nils Hilding;male;27;0;0;350408;7,8542;;S;;;
+3;0;Jussila, Miss. Katriina;female;20;1;0;4136;9,825;;S;;;
+3;0;Jussila, Miss. Mari Aina;female;21;1;0;4137;9,825;;S;;;
+3;1;Jussila, Mr. Eiriik;male;32;0;0;STON/O 2. 3101286;7,925;;S;15;;
+3;0;Kallio, Mr. Nikolai Erland;male;17;0;0;STON/O 2. 3101274;7,125;;S;;;
+3;0;Kalvik, Mr. Johannes Halvorsen;male;21;0;0;8475;8,4333;;S;;;
+3;0;Karaic, Mr. Milan;male;30;0;0;349246;7,8958;;S;;;
+3;1;Karlsson, Mr. Einar Gervasius;male;21;0;0;350053;7,7958;;S;13;;
+3;0;Karlsson, Mr. Julius Konrad Eugen;male;33;0;0;347465;7,8542;;S;;;
+3;0;Karlsson, Mr. Nils August;male;22;0;0;350060;7,5208;;S;;;
+3;1;Karun, Miss. Manca;female;4;0;1;349256;13,4167;;C;15;;
+3;1;Karun, Mr. Franz;male;39;0;1;349256;13,4167;;C;15;;
+3;0;Kassem, Mr. Fared;male;;0;0;2700;7,2292;;C;;;
+3;0;"Katavelas, Mr. Vassilios (""Catavelas Vassilios"")";male;18,5;0;0;2682;7,2292;;C;;58;
+3;0;"Keane, Mr. Andrew ""Andy""";male;;0;0;12460;7,75;;Q;;;
+3;0;Keefe, Mr. Arthur;male;;0;0;323592;7,25;;S;A;;
+3;1;"Kelly, Miss. Anna Katherine ""Annie Kate""";female;;0;0;9234;7,75;;Q;16;;
+3;1;Kelly, Miss. Mary;female;;0;0;14312;7,75;;Q;D;;
+3;0;Kelly, Mr. James;male;34,5;0;0;330911;7,8292;;Q;;70;
+3;0;Kelly, Mr. James;male;44;0;0;363592;8,05;;S;;;
+3;1;Kennedy, Mr. John;male;;0;0;368783;7,75;;Q;;;
+3;0;Khalil, Mr. Betros;male;;1;0;2660;14,4542;;C;;;
+3;0;"Khalil, Mrs. Betros (Zahie ""Maria"" Elias)";female;;1;0;2660;14,4542;;C;;;
+3;0;Kiernan, Mr. John;male;;1;0;367227;7,75;;Q;;;
+3;0;Kiernan, Mr. Philip;male;;1;0;367229;7,75;;Q;;;
+3;0;Kilgannon, Mr. Thomas J;male;;0;0;36865;7,7375;;Q;;;
+3;0;Kink, Miss. Maria;female;22;2;0;315152;8,6625;;S;;;
+3;0;Kink, Mr. Vincenz;male;26;2;0;315151;8,6625;;S;;;
+3;1;Kink-Heilmann, Miss. Luise Gretchen;female;4;0;2;315153;22,025;;S;2;;
+3;1;Kink-Heilmann, Mr. Anton;male;29;3;1;315153;22,025;;S;2;;
+3;1;Kink-Heilmann, Mrs. Anton (Luise Heilmann);female;26;1;1;315153;22,025;;S;2;;
+3;0;Klasen, Miss. Gertrud Emilia;female;1;1;1;350405;12,1833;;S;;;
+3;0;Klasen, Mr. Klas Albin;male;18;1;1;350404;7,8542;;S;;;
+3;0;Klasen, Mrs. (Hulda Kristina Eugenia Lofqvist);female;36;0;2;350405;12,1833;;S;;;
+3;0;Kraeff, Mr. Theodor;male;;0;0;349253;7,8958;;C;;;
+3;1;Krekorian, Mr. Neshan;male;25;0;0;2654;7,2292;F E57;C;10;;
+3;0;Lahoud, Mr. Sarkis;male;;0;0;2624;7,225;;C;;;
+3;0;Laitinen, Miss. Kristina Sofia;female;37;0;0;4135;9,5875;;S;;;
+3;0;Laleff, Mr. Kristo;male;;0;0;349217;7,8958;;S;;;
+3;1;Lam, Mr. Ali;male;;0;0;1601;56,4958;;S;C;;
+3;0;Lam, Mr. Len;male;;0;0;1601;56,4958;;S;;;
+3;1;Landergren, Miss. Aurora Adelia;female;22;0;0;C 7077;7,25;;S;13;;
+3;0;Lane, Mr. Patrick;male;;0;0;7935;7,75;;Q;;;
+3;1;Lang, Mr. Fang;male;26;0;0;1601;56,4958;;S;14;;
+3;0;Larsson, Mr. August Viktor;male;29;0;0;7545;9,4833;;S;;;
+3;0;Larsson, Mr. Bengt Edvin;male;29;0;0;347067;7,775;;S;;;
+3;0;Larsson-Rondberg, Mr. Edvard A;male;22;0;0;347065;7,775;;S;;;
+3;1;"Leeni, Mr. Fahim (""Philip Zenni"")";male;22;0;0;2620;7,225;;C;6;;
+3;0;Lefebre, Master. Henry Forbes;male;;3;1;4133;25,4667;;S;;;
+3;0;Lefebre, Miss. Ida;female;;3;1;4133;25,4667;;S;;;
+3;0;Lefebre, Miss. Jeannie;female;;3;1;4133;25,4667;;S;;;
+3;0;Lefebre, Miss. Mathilde;female;;3;1;4133;25,4667;;S;;;
+3;0;Lefebre, Mrs. Frank (Frances);female;;0;4;4133;25,4667;;S;;;
+3;0;Leinonen, Mr. Antti Gustaf;male;32;0;0;STON/O 2. 3101292;7,925;;S;;;
+3;0;Lemberopolous, Mr. Peter L;male;34,5;0;0;2683;6,4375;;C;;196;
+3;0;Lennon, Miss. Mary;female;;1;0;370371;15,5;;Q;;;
+3;0;Lennon, Mr. Denis;male;;1;0;370371;15,5;;Q;;;
+3;0;Leonard, Mr. Lionel;male;36;0;0;LINE;0;;S;;;
+3;0;Lester, Mr. James;male;39;0;0;A/4 48871;24,15;;S;;;
+3;0;Lievens, Mr. Rene Aime;male;24;0;0;345781;9,5;;S;;;
+3;0;Lindahl, Miss. Agda Thorilda Viktoria;female;25;0;0;347071;7,775;;S;;;
+3;0;Lindblom, Miss. Augusta Charlotta;female;45;0;0;347073;7,75;;S;;;
+3;0;Lindell, Mr. Edvard Bengtsson;male;36;1;0;349910;15,55;;S;A;;
+3;0;Lindell, Mrs. Edvard Bengtsson (Elin Gerda Persson);female;30;1;0;349910;15,55;;S;A;;
+3;1;Lindqvist, Mr. Eino William;male;20;1;0;STON/O 2. 3101285;7,925;;S;15;;
+3;0;Linehan, Mr. Michael;male;;0;0;330971;7,8792;;Q;;;
+3;0;Ling, Mr. Lee;male;28;0;0;1601;56,4958;;S;;;
+3;0;Lithman, Mr. Simon;male;;0;0;S.O./P.P. 251;7,55;;S;;;
+3;0;Lobb, Mr. William Arthur;male;30;1;0;A/5. 3336;16,1;;S;;;
+3;0;Lobb, Mrs. William Arthur (Cordelia K Stanlick);female;26;1;0;A/5. 3336;16,1;;S;;;
+3;0;Lockyer, Mr. Edward;male;;0;0;1222;7,8792;;S;;153;
+3;0;"Lovell, Mr. John Hall (""Henry"")";male;20,5;0;0;A/5 21173;7,25;;S;;;
+3;1;Lulic, Mr. Nikola;male;27;0;0;315098;8,6625;;S;15;;
+3;0;Lundahl, Mr. Johan Svensson;male;51;0;0;347743;7,0542;;S;;;
+3;1;Lundin, Miss. Olga Elida;female;23;0;0;347469;7,8542;;S;10;;
+3;1;Lundstrom, Mr. Thure Edvin;male;32;0;0;350403;7,5792;;S;15;;
+3;0;Lyntakoff, Mr. Stanko;male;;0;0;349235;7,8958;;S;;;
+3;0;MacKay, Mr. George William;male;;0;0;C.A. 42795;7,55;;S;;;
+3;1;"Madigan, Miss. Margaret ""Maggie""";female;;0;0;370370;7,75;;Q;15;;
+3;1;Madsen, Mr. Fridtjof Arne;male;24;0;0;C 17369;7,1417;;S;13;;
+3;0;Maenpaa, Mr. Matti Alexanteri;male;22;0;0;STON/O 2. 3101275;7,125;;S;;;
+3;0;Mahon, Miss. Bridget Delia;female;;0;0;330924;7,8792;;Q;;;
+3;0;Mahon, Mr. John;male;;0;0;AQ/4 3130;7,75;;Q;;;
+3;0;Maisner, Mr. Simon;male;;0;0;A/S 2816;8,05;;S;;;
+3;0;Makinen, Mr. Kalle Edvard;male;29;0;0;STON/O 2. 3101268;7,925;;S;;;
+3;1;Mamee, Mr. Hanna;male;;0;0;2677;7,2292;;C;15;;
+3;0;Mangan, Miss. Mary;female;30,5;0;0;364850;7,75;;Q;;61;
+3;1;Mannion, Miss. Margareth;female;;0;0;36866;7,7375;;Q;16;;
+3;0;Mardirosian, Mr. Sarkis;male;;0;0;2655;7,2292;F E46;C;;;
+3;0;Markoff, Mr. Marin;male;35;0;0;349213;7,8958;;C;;;
+3;0;Markun, Mr. Johann;male;33;0;0;349257;7,8958;;S;;;
+3;1;Masselmani, Mrs. Fatima;female;;0;0;2649;7,225;;C;C;;
+3;0;Matinoff, Mr. Nicola;male;;0;0;349255;7,8958;;C;;;
+3;1;"McCarthy, Miss. Catherine ""Katie""";female;;0;0;383123;7,75;;Q;15 16;;
+3;1;McCormack, Mr. Thomas Joseph;male;;0;0;367228;7,75;;Q;;;
+3;1;McCoy, Miss. Agnes;female;;2;0;367226;23,25;;Q;16;;
+3;1;McCoy, Miss. Alicia;female;;2;0;367226;23,25;;Q;16;;
+3;1;McCoy, Mr. Bernard;male;;2;0;367226;23,25;;Q;16;;
+3;1;McDermott, Miss. Brigdet Delia;female;;0;0;330932;7,7875;;Q;13;;
+3;0;McEvoy, Mr. Michael;male;;0;0;36568;15,5;;Q;;;
+3;1;McGovern, Miss. Mary;female;;0;0;330931;7,8792;;Q;13;;
+3;1;"McGowan, Miss. Anna ""Annie""";female;15;0;0;330923;8,0292;;Q;;;
+3;0;McGowan, Miss. Katherine;female;35;0;0;9232;7,75;;Q;;;
+3;0;McMahon, Mr. Martin;male;;0;0;370372;7,75;;Q;;;
+3;0;McNamee, Mr. Neal;male;24;1;0;376566;16,1;;S;;;
+3;0;McNamee, Mrs. Neal (Eileen O'Leary);female;19;1;0;376566;16,1;;S;;53;
+3;0;McNeill, Miss. Bridget;female;;0;0;370368;7,75;;Q;;;
+3;0;Meanwell, Miss. (Marion Ogden);female;;0;0;SOTON/O.Q. 392087;8,05;;S;;;
+3;0;Meek, Mrs. Thomas (Annie Louise Rowley);female;;0;0;343095;8,05;;S;;;
+3;0;Meo, Mr. Alfonzo;male;55,5;0;0;A.5. 11206;8,05;;S;;201;
+3;0;Mernagh, Mr. Robert;male;;0;0;368703;7,75;;Q;;;
+3;1;Midtsjo, Mr. Karl Albert;male;21;0;0;345501;7,775;;S;15;;
+3;0;Miles, Mr. Frank;male;;0;0;359306;8,05;;S;;;
+3;0;Mineff, Mr. Ivan;male;24;0;0;349233;7,8958;;S;;;
+3;0;Minkoff, Mr. Lazar;male;21;0;0;349211;7,8958;;S;;;
+3;0;Mionoff, Mr. Stoytcho;male;28;0;0;349207;7,8958;;S;;;
+3;0;Mitkoff, Mr. Mito;male;;0;0;349221;7,8958;;S;;;
+3;1;"Mockler, Miss. Helen Mary ""Ellie""";female;;0;0;330980;7,8792;;Q;16;;
+3;0;Moen, Mr. Sigurd Hansen;male;25;0;0;348123;7,65;F G73;S;;309;
+3;1;Moor, Master. Meier;male;6;0;1;392096;12,475;E121;S;14;;
+3;1;Moor, Mrs. (Beila);female;27;0;1;392096;12,475;E121;S;14;;
+3;0;Moore, Mr. Leonard Charles;male;;0;0;A4. 54510;8,05;;S;;;
+3;1;Moran, Miss. Bertha;female;;1;0;371110;24,15;;Q;16;;
+3;0;Moran, Mr. Daniel J;male;;1;0;371110;24,15;;Q;;;
+3;0;Moran, Mr. James;male;;0;0;330877;8,4583;;Q;;;
+3;0;Morley, Mr. William;male;34;0;0;364506;8,05;;S;;;
+3;0;Morrow, Mr. Thomas Rowan;male;;0;0;372622;7,75;;Q;;;
+3;1;Moss, Mr. Albert Johan;male;;0;0;312991;7,775;;S;B;;
+3;1;Moubarek, Master. Gerios;male;;1;1;2661;15,2458;;C;C;;
+3;1;"Moubarek, Master. Halim Gonios (""William George"")";male;;1;1;2661;15,2458;;C;C;;
+3;1;"Moubarek, Mrs. George (Omine ""Amenia"" Alexander)";female;;0;2;2661;15,2458;;C;C;;
+3;1;Moussa, Mrs. (Mantoura Boulos);female;;0;0;2626;7,2292;;C;;;
+3;0;Moutal, Mr. Rahamin Haim;male;;0;0;374746;8,05;;S;;;
+3;1;"Mullens, Miss. Katherine ""Katie""";female;;0;0;35852;7,7333;;Q;16;;
+3;1;Mulvihill, Miss. Bertha E;female;24;0;0;382653;7,75;;Q;15;;
+3;0;Murdlin, Mr. Joseph;male;;0;0;A./5. 3235;8,05;;S;;;
+3;1;"Murphy, Miss. Katherine ""Kate""";female;;1;0;367230;15,5;;Q;16;;
+3;1;Murphy, Miss. Margaret Jane;female;;1;0;367230;15,5;;Q;16;;
+3;1;Murphy, Miss. Nora;female;;0;0;36568;15,5;;Q;16;;
+3;0;Myhrman, Mr. Pehr Fabian Oliver Malkolm;male;18;0;0;347078;7,75;;S;;;
+3;0;Naidenoff, Mr. Penko;male;22;0;0;349206;7,8958;;S;;;
+3;1;"Najib, Miss. Adele Kiamie ""Jane""";female;15;0;0;2667;7,225;;C;C;;
+3;1;"Nakid, Miss. Maria (""Mary"")";female;1;0;2;2653;15,7417;;C;C;;
+3;1;Nakid, Mr. Sahid;male;20;1;1;2653;15,7417;;C;C;;
+3;1;"Nakid, Mrs. Said (Waika ""Mary"" Mowad)";female;19;1;1;2653;15,7417;;C;C;;
+3;0;Nancarrow, Mr. William Henry;male;33;0;0;A./5. 3338;8,05;;S;;;
+3;0;Nankoff, Mr. Minko;male;;0;0;349218;7,8958;;S;;;
+3;0;Nasr, Mr. Mustafa;male;;0;0;2652;7,2292;;C;;;
+3;0;Naughton, Miss. Hannah;female;;0;0;365237;7,75;;Q;;;
+3;0;Nenkoff, Mr. Christo;male;;0;0;349234;7,8958;;S;;;
+3;1;Nicola-Yarred, Master. Elias;male;12;1;0;2651;11,2417;;C;C;;
+3;1;Nicola-Yarred, Miss. Jamila;female;14;1;0;2651;11,2417;;C;C;;
+3;0;Nieminen, Miss. Manta Josefina;female;29;0;0;3101297;7,925;;S;;;
+3;0;Niklasson, Mr. Samuel;male;28;0;0;363611;8,05;;S;;;
+3;1;Nilsson, Miss. Berta Olivia;female;18;0;0;347066;7,775;;S;D;;
+3;1;Nilsson, Miss. Helmina Josefina;female;26;0;0;347470;7,8542;;S;13;;
+3;0;Nilsson, Mr. August Ferdinand;male;21;0;0;350410;7,8542;;S;;;
+3;0;Nirva, Mr. Iisakki Antino Aijo;male;41;0;0;SOTON/O2 3101272;7,125;;S;;;Finland Sudbury, ON
+3;1;Niskanen, Mr. Juha;male;39;0;0;STON/O 2. 3101289;7,925;;S;9;;
+3;0;Nosworthy, Mr. Richard Cater;male;21;0;0;A/4. 39886;7,8;;S;;;
+3;0;Novel, Mr. Mansouer;male;28,5;0;0;2697;7,2292;;C;;181;
+3;1;Nysten, Miss. Anna Sofia;female;22;0;0;347081;7,75;;S;13;;
+3;0;Nysveen, Mr. Johan Hansen;male;61;0;0;345364;6,2375;;S;;;
+3;0;O'Brien, Mr. Thomas;male;;1;0;370365;15,5;;Q;;;
+3;0;O'Brien, Mr. Timothy;male;;0;0;330979;7,8292;;Q;;;
+3;1;"O'Brien, Mrs. Thomas (Johanna ""Hannah"" Godfrey)";female;;1;0;370365;15,5;;Q;;;
+3;0;O'Connell, Mr. Patrick D;male;;0;0;334912;7,7333;;Q;;;
+3;0;O'Connor, Mr. Maurice;male;;0;0;371060;7,75;;Q;;;
+3;0;O'Connor, Mr. Patrick;male;;0;0;366713;7,75;;Q;;;
+3;0;Odahl, Mr. Nils Martin;male;23;0;0;7267;9,225;;S;;;
+3;0;O'Donoghue, Ms. Bridget;female;;0;0;364856;7,75;;Q;;;
+3;1;O'Driscoll, Miss. Bridget;female;;0;0;14311;7,75;;Q;D;;
+3;1;"O'Dwyer, Miss. Ellen ""Nellie""";female;;0;0;330959;7,8792;;Q;;;
+3;1;Ohman, Miss. Velin;female;22;0;0;347085;7,775;;S;C;;
+3;1;O'Keefe, Mr. Patrick;male;;0;0;368402;7,75;;Q;B;;
+3;1;"O'Leary, Miss. Hanora ""Norah""";female;;0;0;330919;7,8292;;Q;13;;
+3;1;Olsen, Master. Artur Karl;male;9;0;1;C 17368;3,1708;;S;13;;
+3;0;Olsen, Mr. Henry Margido;male;28;0;0;C 4001;22,525;;S;;173;
+3;0;Olsen, Mr. Karl Siegwart Andreas;male;42;0;1;4579;8,4042;;S;;;
+3;0;Olsen, Mr. Ole Martin;male;;0;0;Fa 265302;7,3125;;S;;;
+3;0;Olsson, Miss. Elina;female;31;0;0;350407;7,8542;;S;;;
+3;0;Olsson, Mr. Nils Johan Goransson;male;28;0;0;347464;7,8542;;S;;;
+3;1;Olsson, Mr. Oscar Wilhelm;male;32;0;0;347079;7,775;;S;A;;
+3;0;Olsvigen, Mr. Thor Anderson;male;20;0;0;6563;9,225;;S;;89;Oslo, Norway Cameron, WI
+3;0;Oreskovic, Miss. Jelka;female;23;0;0;315085;8,6625;;S;;;
+3;0;Oreskovic, Miss. Marija;female;20;0;0;315096;8,6625;;S;;;
+3;0;Oreskovic, Mr. Luka;male;20;0;0;315094;8,6625;;S;;;
+3;0;Osen, Mr. Olaf Elon;male;16;0;0;7534;9,2167;;S;;;
+3;1;Osman, Mrs. Mara;female;31;0;0;349244;8,6833;;S;;;
+3;0;O'Sullivan, Miss. Bridget Mary;female;;0;0;330909;7,6292;;Q;;;
+3;0;Palsson, Master. Gosta Leonard;male;2;3;1;349909;21,075;;S;;4;
+3;0;Palsson, Master. Paul Folke;male;6;3;1;349909;21,075;;S;;;
+3;0;Palsson, Miss. Stina Viola;female;3;3;1;349909;21,075;;S;;;
+3;0;Palsson, Miss. Torborg Danira;female;8;3;1;349909;21,075;;S;;;
+3;0;Palsson, Mrs. Nils (Alma Cornelia Berglund);female;29;0;4;349909;21,075;;S;;206;
+3;0;Panula, Master. Eino Viljami;male;1;4;1;3101295;39,6875;;S;;;
+3;0;Panula, Master. Juha Niilo;male;7;4;1;3101295;39,6875;;S;;;
+3;0;Panula, Master. Urho Abraham;male;2;4;1;3101295;39,6875;;S;;;
+3;0;Panula, Mr. Ernesti Arvid;male;16;4;1;3101295;39,6875;;S;;;
+3;0;Panula, Mr. Jaako Arnold;male;14;4;1;3101295;39,6875;;S;;;
+3;0;Panula, Mrs. Juha (Maria Emilia Ojala);female;41;0;5

<TRUNCATED>

[02/50] [abbrv] ignite git commit: IGNITE-10335: move ML examples datasets files to resources

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/resources/datasets/titanic_10_rows.csv
----------------------------------------------------------------------
diff --git a/examples/src/main/resources/datasets/titanic_10_rows.csv b/examples/src/main/resources/datasets/titanic_10_rows.csv
new file mode 100644
index 0000000..8146db3
--- /dev/null
+++ b/examples/src/main/resources/datasets/titanic_10_rows.csv
@@ -0,0 +1,11 @@
+pclass;survived;name;sex;age;sibsp;parch;ticket;fare;cabin;embarked;boat;body;homedest
+1;1;Allen, Miss. Elisabeth Walton;;29;;;24160;211,3375;B5;;2;;St Louis, MO
+1;1;Allison, Master. Hudson Trevor;male;0,9167;1;2;113781;151,55;C22 C26;S;11;;Montreal, PQ / Chesterville, ON
+1;0;Allison, Miss. Helen Loraine;female;2;1;2;113781;151,55;C22 C26;S;;;Montreal, PQ / Chesterville, ON
+1;0;Allison, Mr. Hudson Joshua Creighton;male;30;1;2;113781;151,55;C22 C26;S;;135;Montreal, PQ / Chesterville, ON
+1;0;Allison, Mrs. Hudson J C (Bessie Waldo Daniels);female;25;1;2;113781;151,55;C22 C26;S;;;Montreal, PQ / Chesterville, ON
+1;1;Anderson, Mr. Harry;male;48;0;0;19952;26,55;E12;S;3;;New York, NY
+1;1;Andrews, Miss. Kornelia Theodosia;female;63;1;0;13502;77,9583;D7;S;10;;Hudson, NY
+1;0;Andrews, Mr. Thomas Jr;male;39;0;0;112050;0;A36;S;;;Belfast, NI
+1;1;Appleton, Mrs. Edward Dale (Charlotte Lamson);female;53;2;0;11769;51,4792;C101;S;D;;Bayside, Queens, NY
+1;0;Artagaveytia, Mr. Ramon;male;71;0;0;PC 17609;49,5042;;C;;22;Montevideo, Uruguay

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/resources/datasets/two_classed_iris.csv
----------------------------------------------------------------------
diff --git a/examples/src/main/resources/datasets/two_classed_iris.csv b/examples/src/main/resources/datasets/two_classed_iris.csv
new file mode 100644
index 0000000..872c699
--- /dev/null
+++ b/examples/src/main/resources/datasets/two_classed_iris.csv
@@ -0,0 +1,100 @@
+0	5.1	3.5	1.4	0.2
+0	4.9	3	1.4	0.2
+0	4.7	3.2	1.3	0.2
+0	4.6	3.1	1.5	0.2
+0	5	3.6	1.4	0.2
+0	5.4	3.9	1.7	0.4
+0	4.6	3.4	1.4	0.3
+0	5	3.4	1.5	0.2
+0	4.4	2.9	1.4	0.2
+0	4.9	3.1	1.5	0.1
+0	5.4	3.7	1.5	0.2
+0	4.8	3.4	1.6	0.2
+0	4.8	3	1.4	0.1
+0	4.3	3	1.1	0.1
+0	5.8	4	1.2	0.2
+0	5.7	4.4	1.5	0.4
+0	5.4	3.9	1.3	0.4
+0	5.1	3.5	1.4	0.3
+0	5.7	3.8	1.7	0.3
+0	5.1	3.8	1.5	0.3
+0	5.4	3.4	1.7	0.2
+0	5.1	3.7	1.5	0.4
+0	4.6	3.6	1	0.2
+0	5.1	3.3	1.7	0.5
+0	4.8	3.4	1.9	0.2
+0	5	3	1.6	0.2
+0	5	3.4	1.6	0.4
+0	5.2	3.5	1.5	0.2
+0	5.2	3.4	1.4	0.2
+0	4.7	3.2	1.6	0.2
+0	4.8	3.1	1.6	0.2
+0	5.4	3.4	1.5	0.4
+0	5.2	4.1	1.5	0.1
+0	5.5	4.2	1.4	0.2
+0	4.9	3.1	1.5	0.1
+0	5	3.2	1.2	0.2
+0	5.5	3.5	1.3	0.2
+0	4.9	3.1	1.5	0.1
+0	4.4	3	1.3	0.2
+0	5.1	3.4	1.5	0.2
+0	5	3.5	1.3	0.3
+0	4.5	2.3	1.3	0.3
+0	4.4	3.2	1.3	0.2
+0	5	3.5	1.6	0.6
+0	5.1	3.8	1.9	0.4
+0	4.8	3	1.4	0.3
+0	5.1	3.8	1.6	0.2
+0	4.6	3.2	1.4	0.2
+0	5.3	3.7	1.5	0.2
+0	5	3.3	1.4	0.2
+1	7	3.2	4.7	1.4
+1	6.4	3.2	4.5	1.5
+1	6.9	3.1	4.9	1.5
+1	5.5	2.3	4	1.3
+1	6.5	2.8	4.6	1.5
+1	5.7	2.8	4.5	1.3
+1	6.3	3.3	4.7	1.6
+1	4.9	2.4	3.3	1
+1	6.6	2.9	4.6	1.3
+1	5.2	2.7	3.9	1.4
+1	5	2	3.5	1
+1	5.9	3	4.2	1.5
+1	6	2.2	4	1
+1	6.1	2.9	4.7	1.4
+1	5.6	2.9	3.6	1.3
+1	6.7	3.1	4.4	1.4
+1	5.6	3	4.5	1.5
+1	5.8	2.7	4.1	1
+1	6.2	2.2	4.5	1.5
+1	5.6	2.5	3.9	1.1
+1	5.9	3.2	4.8	1.8
+1	6.1	2.8	4	1.3
+1	6.3	2.5	4.9	1.5
+1	6.1	2.8	4.7	1.2
+1	6.4	2.9	4.3	1.3
+1	6.6	3	4.4	1.4
+1	6.8	2.8	4.8	1.4
+1	6.7	3	5	1.7
+1	6	2.9	4.5	1.5
+1	5.7	2.6	3.5	1
+1	5.5	2.4	3.8	1.1
+1	5.5	2.4	3.7	1
+1	5.8	2.7	3.9	1.2
+1	6	2.7	5.1	1.6
+1	5.4	3	4.5	1.5
+1	6	3.4	4.5	1.6
+1	6.7	3.1	4.7	1.5
+1	6.3	2.3	4.4	1.3
+1	5.6	3	4.1	1.3
+1	5.5	2.5	4	1.3
+1	5.5	2.6	4.4	1.2
+1	6.1	3	4.6	1.4
+1	5.8	2.6	4	1.2
+1	5	2.3	3.3	1
+1	5.6	2.7	4.2	1.3
+1	5.7	3	4.2	1.2
+1	5.7	2.9	4.2	1.3
+1	6.2	2.9	4.3	1.3
+1	5.1	2.5	3	1.1
+1	5.7	2.8	4.1	1.3

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/resources/datasets/wine.txt
----------------------------------------------------------------------
diff --git a/examples/src/main/resources/datasets/wine.txt b/examples/src/main/resources/datasets/wine.txt
new file mode 100644
index 0000000..a0b3962
--- /dev/null
+++ b/examples/src/main/resources/datasets/wine.txt
@@ -0,0 +1,178 @@
+1,14.23,1.71,2.43,15.6,127,2.8,3.06,.28,2.29,5.64,1.04,3.92,1065
+1,13.2,1.78,2.14,11.2,100,2.65,2.76,.26,1.28,4.38,1.05,3.4,1050
+1,13.16,2.36,2.67,18.6,101,2.8,3.24,.3,2.81,5.68,1.03,3.17,1185
+1,14.37,1.95,2.5,16.8,113,3.85,3.49,.24,2.18,7.8,.86,3.45,1480
+1,13.24,2.59,2.87,21,118,2.8,2.69,.39,1.82,4.32,1.04,2.93,735
+1,14.2,1.76,2.45,15.2,112,3.27,3.39,.34,1.97,6.75,1.05,2.85,1450
+1,14.39,1.87,2.45,14.6,96,2.5,2.52,.3,1.98,5.25,1.02,3.58,1290
+1,14.06,2.15,2.61,17.6,121,2.6,2.51,.31,1.25,5.05,1.06,3.58,1295
+1,14.83,1.64,2.17,14,97,2.8,2.98,.29,1.98,5.2,1.08,2.85,1045
+1,13.86,1.35,2.27,16,98,2.98,3.15,.22,1.85,7.22,1.01,3.55,1045
+1,14.1,2.16,2.3,18,105,2.95,3.32,.22,2.38,5.75,1.25,3.17,1510
+1,14.12,1.48,2.32,16.8,95,2.2,2.43,.26,1.57,5,1.17,2.82,1280
+1,13.75,1.73,2.41,16,89,2.6,2.76,.29,1.81,5.6,1.15,2.9,1320
+1,14.75,1.73,2.39,11.4,91,3.1,3.69,.43,2.81,5.4,1.25,2.73,1150
+1,14.38,1.87,2.38,12,102,3.3,3.64,.29,2.96,7.5,1.2,3,1547
+1,13.63,1.81,2.7,17.2,112,2.85,2.91,.3,1.46,7.3,1.28,2.88,1310
+1,14.3,1.92,2.72,20,120,2.8,3.14,.33,1.97,6.2,1.07,2.65,1280
+1,13.83,1.57,2.62,20,115,2.95,3.4,.4,1.72,6.6,1.13,2.57,1130
+1,14.19,1.59,2.48,16.5,108,3.3,3.93,.32,1.86,8.7,1.23,2.82,1680
+1,13.64,3.1,2.56,15.2,116,2.7,3.03,.17,1.66,5.1,.96,3.36,845
+1,14.06,1.63,2.28,16,126,3,3.17,.24,2.1,5.65,1.09,3.71,780
+1,12.93,3.8,2.65,18.6,102,2.41,2.41,.25,1.98,4.5,1.03,3.52,770
+1,13.71,1.86,2.36,16.6,101,2.61,2.88,.27,1.69,3.8,1.11,4,1035
+1,12.85,1.6,2.52,17.8,95,2.48,2.37,.26,1.46,3.93,1.09,3.63,1015
+1,13.5,1.81,2.61,20,96,2.53,2.61,.28,1.66,3.52,1.12,3.82,845
+1,13.05,2.05,3.22,25,124,2.63,2.68,.47,1.92,3.58,1.13,3.2,830
+1,13.39,1.77,2.62,16.1,93,2.85,2.94,.34,1.45,4.8,.92,3.22,1195
+1,13.3,1.72,2.14,17,94,2.4,2.19,.27,1.35,3.95,1.02,2.77,1285
+1,13.87,1.9,2.8,19.4,107,2.95,2.97,.37,1.76,4.5,1.25,3.4,915
+1,14.02,1.68,2.21,16,96,2.65,2.33,.26,1.98,4.7,1.04,3.59,1035
+1,13.73,1.5,2.7,22.5,101,3,3.25,.29,2.38,5.7,1.19,2.71,1285
+1,13.58,1.66,2.36,19.1,106,2.86,3.19,.22,1.95,6.9,1.09,2.88,1515
+1,13.68,1.83,2.36,17.2,104,2.42,2.69,.42,1.97,3.84,1.23,2.87,990
+1,13.76,1.53,2.7,19.5,132,2.95,2.74,.5,1.35,5.4,1.25,3,1235
+1,13.51,1.8,2.65,19,110,2.35,2.53,.29,1.54,4.2,1.1,2.87,1095
+1,13.48,1.81,2.41,20.5,100,2.7,2.98,.26,1.86,5.1,1.04,3.47,920
+1,13.28,1.64,2.84,15.5,110,2.6,2.68,.34,1.36,4.6,1.09,2.78,880
+1,13.05,1.65,2.55,18,98,2.45,2.43,.29,1.44,4.25,1.12,2.51,1105
+1,13.07,1.5,2.1,15.5,98,2.4,2.64,.28,1.37,3.7,1.18,2.69,1020
+1,14.22,3.99,2.51,13.2,128,3,3.04,.2,2.08,5.1,.89,3.53,760
+1,13.56,1.71,2.31,16.2,117,3.15,3.29,.34,2.34,6.13,.95,3.38,795
+1,13.41,3.84,2.12,18.8,90,2.45,2.68,.27,1.48,4.28,.91,3,1035
+1,13.88,1.89,2.59,15,101,3.25,3.56,.17,1.7,5.43,.88,3.56,1095
+1,13.24,3.98,2.29,17.5,103,2.64,2.63,.32,1.66,4.36,.82,3,680
+1,13.05,1.77,2.1,17,107,3,3,.28,2.03,5.04,.88,3.35,885
+1,14.21,4.04,2.44,18.9,111,2.85,2.65,.3,1.25,5.24,.87,3.33,1080
+1,14.38,3.59,2.28,16,102,3.25,3.17,.27,2.19,4.9,1.04,3.44,1065
+1,13.9,1.68,2.12,16,101,3.1,3.39,.21,2.14,6.1,.91,3.33,985
+1,14.1,2.02,2.4,18.8,103,2.75,2.92,.32,2.38,6.2,1.07,2.75,1060
+1,13.94,1.73,2.27,17.4,108,2.88,3.54,.32,2.08,8.90,1.12,3.1,1260
+1,13.05,1.73,2.04,12.4,92,2.72,3.27,.17,2.91,7.2,1.12,2.91,1150
+1,13.83,1.65,2.6,17.2,94,2.45,2.99,.22,2.29,5.6,1.24,3.37,1265
+1,13.82,1.75,2.42,14,111,3.88,3.74,.32,1.87,7.05,1.01,3.26,1190
+1,13.77,1.9,2.68,17.1,115,3,2.79,.39,1.68,6.3,1.13,2.93,1375
+1,13.74,1.67,2.25,16.4,118,2.6,2.9,.21,1.62,5.85,.92,3.2,1060
+1,13.56,1.73,2.46,20.5,116,2.96,2.78,.2,2.45,6.25,.98,3.03,1120
+1,14.22,1.7,2.3,16.3,118,3.2,3,.26,2.03,6.38,.94,3.31,970
+1,13.29,1.97,2.68,16.8,102,3,3.23,.31,1.66,6,1.07,2.84,1270
+1,13.72,1.43,2.5,16.7,108,3.4,3.67,.19,2.04,6.8,.89,2.87,1285
+2,12.37,.94,1.36,10.6,88,1.98,.57,.28,.42,1.95,1.05,1.82,520
+2,12.33,1.1,2.28,16,101,2.05,1.09,.63,.41,3.27,1.25,1.67,680
+2,12.64,1.36,2.02,16.8,100,2.02,1.41,.53,.62,5.75,.98,1.59,450
+2,13.67,1.25,1.92,18,94,2.1,1.79,.32,.73,3.8,1.23,2.46,630
+2,12.37,1.13,2.16,19,87,3.5,3.1,.19,1.87,4.45,1.22,2.87,420
+2,12.17,1.45,2.53,19,104,1.89,1.75,.45,1.03,2.95,1.45,2.23,355
+2,12.37,1.21,2.56,18.1,98,2.42,2.65,.37,2.08,4.6,1.19,2.3,678
+2,13.11,1.01,1.7,15,78,2.98,3.18,.26,2.28,5.3,1.12,3.18,502
+2,12.37,1.17,1.92,19.6,78,2.11,2,.27,1.04,4.68,1.12,3.48,510
+2,13.34,.94,2.36,17,110,2.53,1.3,.55,.42,3.17,1.02,1.93,750
+2,12.21,1.19,1.75,16.8,151,1.85,1.28,.14,2.5,2.85,1.28,3.07,718
+2,12.29,1.61,2.21,20.4,103,1.1,1.02,.37,1.46,3.05,.906,1.82,870
+2,13.86,1.51,2.67,25,86,2.95,2.86,.21,1.87,3.38,1.36,3.16,410
+2,13.49,1.66,2.24,24,87,1.88,1.84,.27,1.03,3.74,.98,2.78,472
+2,12.99,1.67,2.6,30,139,3.3,2.89,.21,1.96,3.35,1.31,3.5,985
+2,11.96,1.09,2.3,21,101,3.38,2.14,.13,1.65,3.21,.99,3.13,886
+2,11.66,1.88,1.92,16,97,1.61,1.57,.34,1.15,3.8,1.23,2.14,428
+2,13.03,.9,1.71,16,86,1.95,2.03,.24,1.46,4.6,1.19,2.48,392
+2,11.84,2.89,2.23,18,112,1.72,1.32,.43,.95,2.65,.96,2.52,500
+2,12.33,.99,1.95,14.8,136,1.9,1.85,.35,2.76,3.4,1.06,2.31,750
+2,12.7,3.87,2.4,23,101,2.83,2.55,.43,1.95,2.57,1.19,3.13,463
+2,12,.92,2,19,86,2.42,2.26,.3,1.43,2.5,1.38,3.12,278
+2,12.72,1.81,2.2,18.8,86,2.2,2.53,.26,1.77,3.9,1.16,3.14,714
+2,12.08,1.13,2.51,24,78,2,1.58,.4,1.4,2.2,1.31,2.72,630
+2,13.05,3.86,2.32,22.5,85,1.65,1.59,.61,1.62,4.8,.84,2.01,515
+2,11.84,.89,2.58,18,94,2.2,2.21,.22,2.35,3.05,.79,3.08,520
+2,12.67,.98,2.24,18,99,2.2,1.94,.3,1.46,2.62,1.23,3.16,450
+2,12.16,1.61,2.31,22.8,90,1.78,1.69,.43,1.56,2.45,1.33,2.26,495
+2,11.65,1.67,2.62,26,88,1.92,1.61,.4,1.34,2.6,1.36,3.21,562
+2,11.64,2.06,2.46,21.6,84,1.95,1.69,.48,1.35,2.8,1,2.75,680
+2,12.08,1.33,2.3,23.6,70,2.2,1.59,.42,1.38,1.74,1.07,3.21,625
+2,12.08,1.83,2.32,18.5,81,1.6,1.5,.52,1.64,2.4,1.08,2.27,480
+2,12,1.51,2.42,22,86,1.45,1.25,.5,1.63,3.6,1.05,2.65,450
+2,12.69,1.53,2.26,20.7,80,1.38,1.46,.58,1.62,3.05,.96,2.06,495
+2,12.29,2.83,2.22,18,88,2.45,2.25,.25,1.99,2.15,1.15,3.3,290
+2,11.62,1.99,2.28,18,98,3.02,2.26,.17,1.35,3.25,1.16,2.96,345
+2,12.47,1.52,2.2,19,162,2.5,2.27,.32,3.28,2.6,1.16,2.63,937
+2,11.81,2.12,2.74,21.5,134,1.6,.99,.14,1.56,2.5,.95,2.26,625
+2,12.29,1.41,1.98,16,85,2.55,2.5,.29,1.77,2.9,1.23,2.74,428
+2,12.37,1.07,2.1,18.5,88,3.52,3.75,.24,1.95,4.5,1.04,2.77,660
+2,12.29,3.17,2.21,18,88,2.85,2.99,.45,2.81,2.3,1.42,2.83,406
+2,12.08,2.08,1.7,17.5,97,2.23,2.17,.26,1.4,3.3,1.27,2.96,710
+2,12.6,1.34,1.9,18.5,88,1.45,1.36,.29,1.35,2.45,1.04,2.77,562
+2,12.34,2.45,2.46,21,98,2.56,2.11,.34,1.31,2.8,.8,3.38,438
+2,11.82,1.72,1.88,19.5,86,2.5,1.64,.37,1.42,2.06,.94,2.44,415
+2,12.51,1.73,1.98,20.5,85,2.2,1.92,.32,1.48,2.94,1.04,3.57,672
+2,12.42,2.55,2.27,22,90,1.68,1.84,.66,1.42,2.7,.86,3.3,315
+2,12.25,1.73,2.12,19,80,1.65,2.03,.37,1.63,3.4,1,3.17,510
+2,12.72,1.75,2.28,22.5,84,1.38,1.76,.48,1.63,3.3,.88,2.42,488
+2,12.22,1.29,1.94,19,92,2.36,2.04,.39,2.08,2.7,.86,3.02,312
+2,11.61,1.35,2.7,20,94,2.74,2.92,.29,2.49,2.65,.96,3.26,680
+2,11.46,3.74,1.82,19.5,107,3.18,2.58,.24,3.58,2.9,.75,2.81,562
+2,12.52,2.43,2.17,21,88,2.55,2.27,.26,1.22,2,.9,2.78,325
+2,11.76,2.68,2.92,20,103,1.75,2.03,.6,1.05,3.8,1.23,2.5,607
+2,11.41,.74,2.5,21,88,2.48,2.01,.42,1.44,3.08,1.1,2.31,434
+2,12.08,1.39,2.5,22.5,84,2.56,2.29,.43,1.04,2.9,.93,3.19,385
+2,11.03,1.51,2.2,21.5,85,2.46,2.17,.52,2.01,1.9,1.71,2.87,407
+2,11.82,1.47,1.99,20.8,86,1.98,1.6,.3,1.53,1.95,.95,3.33,495
+2,12.42,1.61,2.19,22.5,108,2,2.09,.34,1.61,2.06,1.06,2.96,345
+2,12.77,3.43,1.98,16,80,1.63,1.25,.43,.83,3.4,.7,2.12,372
+2,12,3.43,2,19,87,2,1.64,.37,1.87,1.28,.93,3.05,564
+2,11.45,2.4,2.42,20,96,2.9,2.79,.32,1.83,3.25,.8,3.39,625
+2,11.56,2.05,3.23,28.5,119,3.18,5.08,.47,1.87,6,.93,3.69,465
+2,12.42,4.43,2.73,26.5,102,2.2,2.13,.43,1.71,2.08,.92,3.12,365
+2,13.05,5.8,2.13,21.5,86,2.62,2.65,.3,2.01,2.6,.73,3.1,380
+2,11.87,4.31,2.39,21,82,2.86,3.03,.21,2.91,2.8,.75,3.64,380
+2,12.07,2.16,2.17,21,85,2.6,2.65,.37,1.35,2.76,.86,3.28,378
+2,12.43,1.53,2.29,21.5,86,2.74,3.15,.39,1.77,3.94,.69,2.84,352
+2,11.79,2.13,2.78,28.5,92,2.13,2.24,.58,1.76,3,.97,2.44,466
+2,12.37,1.63,2.3,24.5,88,2.22,2.45,.4,1.9,2.12,.89,2.78,342
+2,12.04,4.3,2.38,22,80,2.1,1.75,.42,1.35,2.6,.79,2.57,580
+3,12.86,1.35,2.32,18,122,1.51,1.25,.21,.94,4.1,.76,1.29,630
+3,12.88,2.99,2.4,20,104,1.3,1.22,.24,.83,5.4,.74,1.42,530
+3,12.81,2.31,2.4,24,98,1.15,1.09,.27,.83,5.7,.66,1.36,560
+3,12.7,3.55,2.36,21.5,106,1.7,1.2,.17,.84,5,.78,1.29,600
+3,12.51,1.24,2.25,17.5,85,2,.58,.6,1.25,5.45,.75,1.51,650
+3,12.6,2.46,2.2,18.5,94,1.62,.66,.63,.94,7.1,.73,1.58,695
+3,12.25,4.72,2.54,21,89,1.38,.47,.53,.8,3.85,.75,1.27,720
+3,12.53,5.51,2.64,25,96,1.79,.6,.63,1.1,5,.82,1.69,515
+3,13.49,3.59,2.19,19.5,88,1.62,.48,.58,.88,5.7,.81,1.82,580
+3,12.84,2.96,2.61,24,101,2.32,.6,.53,.81,4.92,.89,2.15,590
+3,12.93,2.81,2.7,21,96,1.54,.5,.53,.75,4.6,.77,2.31,600
+3,13.36,2.56,2.35,20,89,1.4,.5,.37,.64,5.6,.7,2.47,780
+3,13.52,3.17,2.72,23.5,97,1.55,.52,.5,.55,4.35,.89,2.06,520
+3,13.62,4.95,2.35,20,92,2,.8,.47,1.02,4.4,.91,2.05,550
+3,12.25,3.88,2.2,18.5,112,1.38,.78,.29,1.14,8.21,.65,2,855
+3,13.16,3.57,2.15,21,102,1.5,.55,.43,1.3,4,.6,1.68,830
+3,13.88,5.04,2.23,20,80,.98,.34,.4,.68,4.9,.58,1.33,415
+3,12.87,4.61,2.48,21.5,86,1.7,.65,.47,.86,7.65,.54,1.86,625
+3,13.32,3.24,2.38,21.5,92,1.93,.76,.45,1.25,8.42,.55,1.62,650
+3,13.08,3.9,2.36,21.5,113,1.41,1.39,.34,1.14,9.40,.57,1.33,550
+3,13.5,3.12,2.62,24,123,1.4,1.57,.22,1.25,8.60,.59,1.3,500
+3,12.79,2.67,2.48,22,112,1.48,1.36,.24,1.26,10.8,.48,1.47,480
+3,13.11,1.9,2.75,25.5,116,2.2,1.28,.26,1.56,7.1,.61,1.33,425
+3,13.23,3.3,2.28,18.5,98,1.8,.83,.61,1.87,10.52,.56,1.51,675
+3,12.58,1.29,2.1,20,103,1.48,.58,.53,1.4,7.6,.58,1.55,640
+3,13.17,5.19,2.32,22,93,1.74,.63,.61,1.55,7.9,.6,1.48,725
+3,13.84,4.12,2.38,19.5,89,1.8,.83,.48,1.56,9.01,.57,1.64,480
+3,12.45,3.03,2.64,27,97,1.9,.58,.63,1.14,7.5,.67,1.73,880
+3,14.34,1.68,2.7,25,98,2.8,1.31,.53,2.7,13,.57,1.96,660
+3,13.48,1.67,2.64,22.5,89,2.6,1.1,.52,2.29,11.75,.57,1.78,620
+3,12.36,3.83,2.38,21,88,2.3,.92,.5,1.04,7.65,.56,1.58,520
+3,13.69,3.26,2.54,20,107,1.83,.56,.5,.8,5.88,.96,1.82,680
+3,12.85,3.27,2.58,22,106,1.65,.6,.6,.96,5.58,.87,2.11,570
+3,12.96,3.45,2.35,18.5,106,1.39,.7,.4,.94,5.28,.68,1.75,675
+3,13.78,2.76,2.3,22,90,1.35,.68,.41,1.03,9.58,.7,1.68,615
+3,13.73,4.36,2.26,22.5,88,1.28,.47,.52,1.15,6.62,.78,1.75,520
+3,13.45,3.7,2.6,23,111,1.7,.92,.43,1.46,10.68,.85,1.56,695
+3,12.82,3.37,2.3,19.5,88,1.48,.66,.4,.97,10.26,.72,1.75,685
+3,13.58,2.58,2.69,24.5,105,1.55,.84,.39,1.54,8.66,.74,1.8,750
+3,13.4,4.6,2.86,25,112,1.98,.96,.27,1.11,8.5,.67,1.92,630
+3,12.2,3.03,2.32,19,96,1.25,.49,.4,.73,5.5,.66,1.83,510
+3,12.77,2.39,2.28,19.5,86,1.39,.51,.48,.64,9.899999,.57,1.63,470
+3,14.16,2.51,2.48,20,91,1.68,.7,.44,1.24,9.7,.62,1.71,660
+3,13.71,5.65,2.45,20.5,95,1.68,.61,.52,1.06,7.7,.64,1.74,740
+3,13.4,3.91,2.48,23,102,1.8,.75,.43,1.41,7.3,.7,1.56,750
+3,13.27,4.28,2.26,20,120,1.59,.69,.43,1.35,10.2,.59,1.56,835
+3,13.17,2.59,2.37,20,120,1.65,.68,.53,1.46,9.3,.6,1.62,840
+3,14.13,4.1,2.74,24.5,96,2.05,.76,.56,1.35,9.2,.61,1.6,560

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index a11be9f..14f970e 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -873,7 +873,6 @@
                                         <exclude>src/main/java/org/apache/ignite/examples/streaming/wordcount/*.txt</exclude><!--books examples-->
                                         <exclude>examples/src/main/java/org/apache/ignite/examples/streaming/wordcount/*.txt</exclude><!--books examples-->
                                         <exclude>**/resources/datasets/**/*</exclude><!--Datasets in ml module-->
-                                        <exclude>**/ml/util/datasets/**/*</exclude><!--Datasets in ml examples-->
                                         <exclude>src/main/java/org/jetbrains/annotations/*.java</exclude><!--copyright-->
                                         <exclude>dev-tools/IGNITE-*.patch</exclude>
                                         <exclude>dev-tools/.gradle/**/*</exclude>


[06/50] [abbrv] ignite git commit: IGNITE-10335: move ML examples datasets files to resources

Posted by ag...@apache.org.
IGNITE-10335: move ML examples datasets files to resources

This closes #5434


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

Branch: refs/heads/ignite-9720
Commit: 1dea0a4273bf54ca2361a3a940db22ba89274ab5
Parents: 39b59d2
Author: Oleg Ignatenko <oi...@gridgain.com>
Authored: Fri Nov 23 16:54:25 2018 +0300
Committer: Yury Babak <yb...@gridgain.com>
Committed: Fri Nov 23 16:54:25 2018 +0300

----------------------------------------------------------------------
 .../TensorFlowDistributedInferenceExample.java  |   16 +-
 .../TensorFlowLocalInferenceExample.java        |   16 +-
 .../TensorFlowThreadedInferenceExample.java     |   16 +-
 .../examples/ml/util/MLSandboxDatasets.java     |   16 +-
 .../ignite/examples/ml/util/SandboxMLCache.java |   11 +-
 .../ignite/examples/ml/util/datasets/README.md  |    2 -
 .../ml/util/datasets/boston_housing_dataset.txt |  505 -------
 .../ml/util/datasets/cleared_machines.csv       |  209 ---
 .../ml/util/datasets/glass_identification.csv   |  116 --
 .../ignite/examples/ml/util/datasets/iris.txt   |  150 --
 .../examples/ml/util/datasets/mortalitydata.csv |   53 -
 .../ml/util/datasets/t10k-images-idx3-ubyte     |  Bin 7840016 -> 0 bytes
 .../ml/util/datasets/t10k-labels-idx1-ubyte     |  Bin 10008 -> 0 bytes
 .../examples/ml/util/datasets/titanic.csv       | 1310 ------------------
 .../ml/util/datasets/titanic_10_rows.csv        |   11 -
 .../ml/util/datasets/two_classed_iris.csv       |  100 --
 .../ignite/examples/ml/util/datasets/wine.txt   |  178 ---
 examples/src/main/resources/datasets/README.md  |    2 +
 .../datasets/boston_housing_dataset.txt         |  505 +++++++
 .../resources/datasets/cleared_machines.csv     |  209 +++
 .../resources/datasets/glass_identification.csv |  116 ++
 examples/src/main/resources/datasets/iris.txt   |  150 ++
 .../main/resources/datasets/mortalitydata.csv   |   53 +
 .../resources/datasets/t10k-images-idx3-ubyte   |  Bin 0 -> 9876027 bytes
 .../resources/datasets/t10k-labels-idx1-ubyte   |  Bin 0 -> 10008 bytes
 .../src/main/resources/datasets/titanic.csv     | 1310 ++++++++++++++++++
 .../main/resources/datasets/titanic_10_rows.csv |   11 +
 .../resources/datasets/two_classed_iris.csv     |  100 ++
 examples/src/main/resources/datasets/wine.txt   |  178 +++
 parent/pom.xml                                  |    1 -
 30 files changed, 2676 insertions(+), 2668 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/java/org/apache/ignite/examples/ml/inference/TensorFlowDistributedInferenceExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/inference/TensorFlowDistributedInferenceExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/inference/TensorFlowDistributedInferenceExample.java
index cc22df3..ec8cac6 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ml/inference/TensorFlowDistributedInferenceExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/inference/TensorFlowDistributedInferenceExample.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Objects;
 import java.util.Random;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
@@ -46,10 +47,10 @@ public class TensorFlowDistributedInferenceExample {
     private static final String MODEL_PATH = "examples/src/main/resources/ml/mnist_tf_model";
 
     /** Path to the MNIST images data. */
-    private static final String MNIST_IMG_PATH = "org/apache/ignite/examples/ml/util/datasets/t10k-images-idx3-ubyte";
+    private static final String MNIST_IMG_PATH = "examples/src/main/resources/datasets/t10k-images-idx3-ubyte";
 
     /** Path to the MNIST labels data. */
-    private static final String MNIST_LBL_PATH = "org/apache/ignite/examples/ml/util/datasets/t10k-labels-idx1-ubyte";
+    private static final String MNIST_LBL_PATH = "examples/src/main/resources/datasets/t10k-labels-idx1-ubyte";
 
     /** Run example. */
     public static void main(String[] args) throws IOException, ExecutionException, InterruptedException {
@@ -69,13 +70,12 @@ public class TensorFlowDistributedInferenceExample {
                     return Tensor.create(reshaped);
                 })
 
-                .withOutput(Collections.singletonList("ArgMax"), collectedTensors -> {
-                    return collectedTensors.get("ArgMax").copyTo(new long[1])[0];
-                });
+                .withOutput(Collections.singletonList("ArgMax"), collectedTensors -> collectedTensors.get("ArgMax")
+                    .copyTo(new long[1])[0]);
 
-            List<MnistUtils.MnistLabeledImage> images = MnistUtils.mnistAsListFromResource(
-                MNIST_IMG_PATH,
-                MNIST_LBL_PATH,
+            List<MnistUtils.MnistLabeledImage> images = MnistUtils.mnistAsList(
+                Objects.requireNonNull(IgniteUtils.resolveIgnitePath(MNIST_IMG_PATH)).getPath(),
+                Objects.requireNonNull(IgniteUtils.resolveIgnitePath(MNIST_LBL_PATH)).getPath(),
                 new Random(0),
                 10000
             );

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/java/org/apache/ignite/examples/ml/inference/TensorFlowLocalInferenceExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/inference/TensorFlowLocalInferenceExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/inference/TensorFlowLocalInferenceExample.java
index fc25c7e..0e79856 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ml/inference/TensorFlowLocalInferenceExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/inference/TensorFlowLocalInferenceExample.java
@@ -21,6 +21,7 @@ import java.io.File;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
+import java.util.Objects;
 import java.util.Random;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.ml.inference.InfModel;
@@ -40,10 +41,10 @@ public class TensorFlowLocalInferenceExample {
     private static final String MODEL_PATH = "examples/src/main/resources/ml/mnist_tf_model";
 
     /** Path to the MNIST images data. */
-    private static final String MNIST_IMG_PATH = "org/apache/ignite/examples/ml/util/datasets/t10k-images-idx3-ubyte";
+    private static final String MNIST_IMG_PATH = "examples/src/main/resources/datasets/t10k-images-idx3-ubyte";
 
     /** Path to the MNIST labels data. */
-    private static final String MNIST_LBL_PATH = "org/apache/ignite/examples/ml/util/datasets/t10k-labels-idx1-ubyte";
+    private static final String MNIST_LBL_PATH = "examples/src/main/resources/datasets/t10k-labels-idx1-ubyte";
 
     /** Run example. */
     public static void main(String[] args) throws IOException {
@@ -60,13 +61,12 @@ public class TensorFlowLocalInferenceExample {
                     reshaped[0][i / 28][i % 28] = (float)doubles[i];
                 return Tensor.create(reshaped);
             })
-            .withOutput(Collections.singletonList("ArgMax"), collectedTensors -> {
-                return collectedTensors.get("ArgMax").copyTo(new long[1])[0];
-            });
+            .withOutput(Collections.singletonList("ArgMax"), collectedTensors -> collectedTensors.get("ArgMax")
+                .copyTo(new long[1])[0]);
 
-        List<MnistUtils.MnistLabeledImage> images = MnistUtils.mnistAsListFromResource(
-            MNIST_IMG_PATH,
-            MNIST_LBL_PATH,
+        List<MnistUtils.MnistLabeledImage> images = MnistUtils.mnistAsList(
+            Objects.requireNonNull(IgniteUtils.resolveIgnitePath(MNIST_IMG_PATH)).getPath(),
+            Objects.requireNonNull(IgniteUtils.resolveIgnitePath(MNIST_LBL_PATH)).getPath(),
             new Random(0),
             10000
         );

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/java/org/apache/ignite/examples/ml/inference/TensorFlowThreadedInferenceExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/inference/TensorFlowThreadedInferenceExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/inference/TensorFlowThreadedInferenceExample.java
index d756016..002e5ae 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ml/inference/TensorFlowThreadedInferenceExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/inference/TensorFlowThreadedInferenceExample.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Objects;
 import java.util.Random;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
@@ -44,10 +45,10 @@ public class TensorFlowThreadedInferenceExample {
     private static final String MODEL_PATH = "examples/src/main/resources/ml/mnist_tf_model";
 
     /** Path to the MNIST images data. */
-    private static final String MNIST_IMG_PATH = "org/apache/ignite/examples/ml/util/datasets/t10k-images-idx3-ubyte";
+    private static final String MNIST_IMG_PATH = "examples/src/main/resources/datasets/t10k-images-idx3-ubyte";
 
     /** Path to the MNIST labels data. */
-    private static final String MNIST_LBL_PATH = "org/apache/ignite/examples/ml/util/datasets/t10k-labels-idx1-ubyte";
+    private static final String MNIST_LBL_PATH = "examples/src/main/resources/datasets/t10k-labels-idx1-ubyte";
 
     /** Run example. */
     public static void main(String[] args) throws IOException, ExecutionException, InterruptedException {
@@ -66,13 +67,12 @@ public class TensorFlowThreadedInferenceExample {
                 return Tensor.create(reshaped);
             })
 
-            .withOutput(Collections.singletonList("ArgMax"), collectedTensors -> {
-                return collectedTensors.get("ArgMax").copyTo(new long[1])[0];
-            });
+            .withOutput(Collections.singletonList("ArgMax"), collectedTensors -> collectedTensors.get("ArgMax")
+                .copyTo(new long[1])[0]);
 
-        List<MnistUtils.MnistLabeledImage> images = MnistUtils.mnistAsListFromResource(
-            MNIST_IMG_PATH,
-            MNIST_LBL_PATH,
+        List<MnistUtils.MnistLabeledImage> images = MnistUtils.mnistAsList(
+            Objects.requireNonNull(IgniteUtils.resolveIgnitePath(MNIST_IMG_PATH)).getPath(),
+            Objects.requireNonNull(IgniteUtils.resolveIgnitePath(MNIST_LBL_PATH)).getPath(),
             new Random(0),
             10000
         );

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/java/org/apache/ignite/examples/ml/util/MLSandboxDatasets.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/util/MLSandboxDatasets.java b/examples/src/main/java/org/apache/ignite/examples/ml/util/MLSandboxDatasets.java
index 800a92f..b7380e0 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ml/util/MLSandboxDatasets.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/util/MLSandboxDatasets.java
@@ -22,35 +22,35 @@ package org.apache.ignite.examples.ml.util;
  */
 public enum MLSandboxDatasets {
     /** The full Iris dataset from Machine Learning Repository. */
-    IRIS("datasets/iris.txt", false, "\t"),
+    IRIS("examples/src/main/resources/datasets/iris.txt", false, "\t"),
 
     /** The Titanic dataset from Kaggle competition. */
-    TITANIC("datasets/titanic.csv", true, ";"),
+    TITANIC("examples/src/main/resources/datasets/titanic.csv", true, ";"),
 
     /** The 1st and 2nd classes from the Iris dataset. */
-    TWO_CLASSED_IRIS("datasets/two_classed_iris.csv", false, "\t"),
+    TWO_CLASSED_IRIS("examples/src/main/resources/datasets/two_classed_iris.csv", false, "\t"),
 
     /** The dataset is about different computers' properties based on https://archive.ics.uci.edu/ml/datasets/Computer+Hardware. */
-    CLEARED_MACHINES("datasets/cleared_machines.csv", false, ";"),
+    CLEARED_MACHINES("examples/src/main/resources/datasets/cleared_machines.csv", false, ";"),
 
     /**
      * The health data is related to death rate based on; doctor availability, hospital availability,
      * annual per capita income, and population density people per square mile.
      */
-    MORTALITY_DATA("datasets/mortalitydata.csv", false, ";"),
+    MORTALITY_DATA("examples/src/main/resources/datasets/mortalitydata.csv", false, ";"),
 
     /**
      * The preprocessed Glass dataset from the Machine Learning Repository https://archive.ics.uci.edu/ml/datasets/Glass+Identification
      * There are 3 classes with labels: 1 {building_windows_float_processed}, 3 {vehicle_windows_float_processed}, 7 {headlamps}.
      * Feature names: 'Na-Sodium', 'Mg-Magnesium', 'Al-Aluminum', 'Ba-Barium', 'Fe-Iron'.
      */
-    GLASS_IDENTIFICATION("datasets/glass_identification.csv", false, ";"),
+    GLASS_IDENTIFICATION("examples/src/main/resources/datasets/glass_identification.csv", false, ";"),
 
     /** The Wine recognition data. Could be found <a href="https://archive.ics.uci.edu/ml/machine-learning-databases/wine/">here</a>. */
-    WINE_RECOGNITION("datasets/wine.txt", false, ","),
+    WINE_RECOGNITION("examples/src/main/resources/datasets/wine.txt", false, ","),
 
     /** The Boston house-prices dataset. Could be found <a href="https://archive.ics.uci.edu/ml/machine-learning-databases/housing/">here</a>. */
-    BOSTON_HOUSE_PRICES("datasets/boston_housing_dataset.txt", false, ",");
+    BOSTON_HOUSE_PRICES("examples/src/main/resources/datasets/boston_housing_dataset.txt", false, ",");
 
     /** Filename. */
     private final String filename;

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/java/org/apache/ignite/examples/ml/util/SandboxMLCache.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/util/SandboxMLCache.java b/examples/src/main/java/org/apache/ignite/examples/ml/util/SandboxMLCache.java
index fa1235b..a8431de 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ml/util/SandboxMLCache.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/util/SandboxMLCache.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.examples.ml.util;
 
+import java.io.File;
 import java.io.FileNotFoundException;
 import java.nio.file.Paths;
 import java.text.NumberFormat;
@@ -28,6 +29,7 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.ml.math.exceptions.knn.FileParsingException;
 import org.apache.ignite.ml.math.primitives.vector.Vector;
 import org.apache.ignite.ml.math.primitives.vector.VectorUtils;
@@ -94,7 +96,14 @@ public class SandboxMLCache {
 
         IgniteCache<Integer, Vector> cache = getCache();
 
-        Scanner scanner = new Scanner(SandboxMLCache.class.getResourceAsStream(dataset.getFileName()));
+        String fileName = dataset.getFileName();
+
+        File file = IgniteUtils.resolveIgnitePath(fileName);
+
+        if (file == null)
+            throw new FileNotFoundException(fileName);
+
+        Scanner scanner = new Scanner(file);
 
         int cnt = 0;
         while (scanner.hasNextLine()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/README.md
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/README.md b/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/README.md
deleted file mode 100644
index 2f9c5ec..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/README.md
+++ /dev/null
@@ -1,2 +0,0 @@
-iris.txt and cleared_machines are from Lichman, M. (2013). UCI Machine Learning Repository [http://archive.ics.uci.edu/ml]. Irvine, CA: University of California, School of Information and Computer Science.
-Read more about machine dataset http://archive.ics.uci.edu/ml/machine-learning-databases/cpu-performance/machine.names
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/boston_housing_dataset.txt
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/boston_housing_dataset.txt b/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/boston_housing_dataset.txt
deleted file mode 100644
index 654a340..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/boston_housing_dataset.txt
+++ /dev/null
@@ -1,505 +0,0 @@
-0.02731,0.00,7.070,0,0.4690,6.4210,78.90,4.9671,2,242.0,17.80,396.90,9.14,21.60
-0.02729,0.00,7.070,0,0.4690,7.1850,61.10,4.9671,2,242.0,17.80,392.83,4.03,34.70
-0.03237,0.00,2.180,0,0.4580,6.9980,45.80,6.0622,3,222.0,18.70,394.63,2.94,33.40
-0.06905,0.00,2.180,0,0.4580,7.1470,54.20,6.0622,3,222.0,18.70,396.90,5.33,36.20
-0.02985,0.00,2.180,0,0.4580,6.4300,58.70,6.0622,3,222.0,18.70,394.12,5.21,28.70
-0.08829,12.50,7.870,0,0.5240,6.0120,66.60,5.5605,5,311.0,15.20,395.60,12.43,22.90
-0.14455,12.50,7.870,0,0.5240,6.1720,96.10,5.9505,5,311.0,15.20,396.90,19.15,27.10
-0.21124,12.50,7.870,0,0.5240,5.6310,100.00,6.0821,5,311.0,15.20,386.63,29.93,16.50
-0.17004,12.50,7.870,0,0.5240,6.0040,85.90,6.5921,5,311.0,15.20,386.71,17.10,18.90
-0.22489,12.50,7.870,0,0.5240,6.3770,94.30,6.3467,5,311.0,15.20,392.52,20.45,15.00
-0.11747,12.50,7.870,0,0.5240,6.0090,82.90,6.2267,5,311.0,15.20,396.90,13.27,18.90
-0.09378,12.50,7.870,0,0.5240,5.8890,39.00,5.4509,5,311.0,15.20,390.50,15.71,21.70
-0.62976,0.00,8.140,0,0.5380,5.9490,61.80,4.7075,4,307.0,21.00,396.90,8.26,20.40
-0.63796,0.00,8.140,0,0.5380,6.0960,84.50,4.4619,4,307.0,21.00,380.02,10.26,18.20
-0.62739,0.00,8.140,0,0.5380,5.8340,56.50,4.4986,4,307.0,21.00,395.62,8.47,19.90
-1.05393,0.00,8.140,0,0.5380,5.9350,29.30,4.4986,4,307.0,21.00,386.85,6.58,23.10
-0.78420,0.00,8.140,0,0.5380,5.9900,81.70,4.2579,4,307.0,21.00,386.75,14.67,17.50
-0.80271,0.00,8.140,0,0.5380,5.4560,36.60,3.7965,4,307.0,21.00,288.99,11.69,20.20
-0.72580,0.00,8.140,0,0.5380,5.7270,69.50,3.7965,4,307.0,21.00,390.95,11.28,18.20
-1.25179,0.00,8.140,0,0.5380,5.5700,98.10,3.7979,4,307.0,21.00,376.57,21.02,13.60
-0.85204,0.00,8.140,0,0.5380,5.9650,89.20,4.0123,4,307.0,21.00,392.53,13.83,19.60
-1.23247,0.00,8.140,0,0.5380,6.1420,91.70,3.9769,4,307.0,21.00,396.90,18.72,15.20
-0.98843,0.00,8.140,0,0.5380,5.8130,100.00,4.0952,4,307.0,21.00,394.54,19.88,14.50
-0.75026,0.00,8.140,0,0.5380,5.9240,94.10,4.3996,4,307.0,21.00,394.33,16.30,15.60
-0.84054,0.00,8.140,0,0.5380,5.5990,85.70,4.4546,4,307.0,21.00,303.42,16.51,13.90
-0.67191,0.00,8.140,0,0.5380,5.8130,90.30,4.6820,4,307.0,21.00,376.88,14.81,16.60
-0.95577,0.00,8.140,0,0.5380,6.0470,88.80,4.4534,4,307.0,21.00,306.38,17.28,14.80
-0.77299,0.00,8.140,0,0.5380,6.4950,94.40,4.4547,4,307.0,21.00,387.94,12.80,18.40
-1.00245,0.00,8.140,0,0.5380,6.6740,87.30,4.2390,4,307.0,21.00,380.23,11.98,21.00
-1.13081,0.00,8.140,0,0.5380,5.7130,94.10,4.2330,4,307.0,21.00,360.17,22.60,12.70
-1.35472,0.00,8.140,0,0.5380,6.0720,100.00,4.1750,4,307.0,21.00,376.73,13.04,14.50
-1.38799,0.00,8.140,0,0.5380,5.9500,82.00,3.9900,4,307.0,21.00,232.60,27.71,13.20
-1.15172,0.00,8.140,0,0.5380,5.7010,95.00,3.7872,4,307.0,21.00,358.77,18.35,13.10
-1.61282,0.00,8.140,0,0.5380,6.0960,96.90,3.7598,4,307.0,21.00,248.31,20.34,13.50
-0.06417,0.00,5.960,0,0.4990,5.9330,68.20,3.3603,5,279.0,19.20,396.90,9.68,18.90
-0.09744,0.00,5.960,0,0.4990,5.8410,61.40,3.3779,5,279.0,19.20,377.56,11.41,20.00
-0.08014,0.00,5.960,0,0.4990,5.8500,41.50,3.9342,5,279.0,19.20,396.90,8.77,21.00
-0.17505,0.00,5.960,0,0.4990,5.9660,30.20,3.8473,5,279.0,19.20,393.43,10.13,24.70
-0.02763,75.00,2.950,0,0.4280,6.5950,21.80,5.4011,3,252.0,18.30,395.63,4.32,30.80
-0.03359,75.00,2.950,0,0.4280,7.0240,15.80,5.4011,3,252.0,18.30,395.62,1.98,34.90
-0.12744,0.00,6.910,0,0.4480,6.7700,2.90,5.7209,3,233.0,17.90,385.41,4.84,26.60
-0.14150,0.00,6.910,0,0.4480,6.1690,6.60,5.7209,3,233.0,17.90,383.37,5.81,25.30
-0.15936,0.00,6.910,0,0.4480,6.2110,6.50,5.7209,3,233.0,17.90,394.46,7.44,24.70
-0.12269,0.00,6.910,0,0.4480,6.0690,40.00,5.7209,3,233.0,17.90,389.39,9.55,21.20
-0.17142,0.00,6.910,0,0.4480,5.6820,33.80,5.1004,3,233.0,17.90,396.90,10.21,19.30
-0.18836,0.00,6.910,0,0.4480,5.7860,33.30,5.1004,3,233.0,17.90,396.90,14.15,20.00
-0.22927,0.00,6.910,0,0.4480,6.0300,85.50,5.6894,3,233.0,17.90,392.74,18.80,16.60
-0.25387,0.00,6.910,0,0.4480,5.3990,95.30,5.8700,3,233.0,17.90,396.90,30.81,14.40
-0.21977,0.00,6.910,0,0.4480,5.6020,62.00,6.0877,3,233.0,17.90,396.90,16.20,19.40
-0.08873,21.00,5.640,0,0.4390,5.9630,45.70,6.8147,4,243.0,16.80,395.56,13.45,19.70
-0.04337,21.00,5.640,0,0.4390,6.1150,63.00,6.8147,4,243.0,16.80,393.97,9.43,20.50
-0.05360,21.00,5.640,0,0.4390,6.5110,21.10,6.8147,4,243.0,16.80,396.90,5.28,25.00
-0.04981,21.00,5.640,0,0.4390,5.9980,21.40,6.8147,4,243.0,16.80,396.90,8.43,23.40
-0.01360,75.00,4.000,0,0.4100,5.8880,47.60,7.3197,3,469.0,21.10,396.90,14.80,18.90
-0.01311,90.00,1.220,0,0.4030,7.2490,21.90,8.6966,5,226.0,17.90,395.93,4.81,35.40
-0.02055,85.00,0.740,0,0.4100,6.3830,35.70,9.1876,2,313.0,17.30,396.90,5.77,24.70
-0.01432,100.00,1.320,0,0.4110,6.8160,40.50,8.3248,5,256.0,15.10,392.90,3.95,31.60
-0.15445,25.00,5.130,0,0.4530,6.1450,29.20,7.8148,8,284.0,19.70,390.68,6.86,23.30
-0.10328,25.00,5.130,0,0.4530,5.9270,47.20,6.9320,8,284.0,19.70,396.90,9.22,19.60
-0.14932,25.00,5.130,0,0.4530,5.7410,66.20,7.2254,8,284.0,19.70,395.11,13.15,18.70
-0.17171,25.00,5.130,0,0.4530,5.9660,93.40,6.8185,8,284.0,19.70,378.08,14.44,16.00
-0.11027,25.00,5.130,0,0.4530,6.4560,67.80,7.2255,8,284.0,19.70,396.90,6.73,22.20
-0.12650,25.00,5.130,0,0.4530,6.7620,43.40,7.9809,8,284.0,19.70,395.58,9.50,25.00
-0.01951,17.50,1.380,0,0.4161,7.1040,59.50,9.2229,3,216.0,18.60,393.24,8.05,33.00
-0.03584,80.00,3.370,0,0.3980,6.2900,17.80,6.6115,4,337.0,16.10,396.90,4.67,23.50
-0.04379,80.00,3.370,0,0.3980,5.7870,31.10,6.6115,4,337.0,16.10,396.90,10.24,19.40
-0.05789,12.50,6.070,0,0.4090,5.8780,21.40,6.4980,4,345.0,18.90,396.21,8.10,22.00
-0.13554,12.50,6.070,0,0.4090,5.5940,36.80,6.4980,4,345.0,18.90,396.90,13.09,17.40
-0.12816,12.50,6.070,0,0.4090,5.8850,33.00,6.4980,4,345.0,18.90,396.90,8.79,20.90
-0.08826,0.00,10.810,0,0.4130,6.4170,6.60,5.2873,4,305.0,19.20,383.73,6.72,24.20
-0.15876,0.00,10.810,0,0.4130,5.9610,17.50,5.2873,4,305.0,19.20,376.94,9.88,21.70
-0.09164,0.00,10.810,0,0.4130,6.0650,7.80,5.2873,4,305.0,19.20,390.91,5.52,22.80
-0.19539,0.00,10.810,0,0.4130,6.2450,6.20,5.2873,4,305.0,19.20,377.17,7.54,23.40
-0.07896,0.00,12.830,0,0.4370,6.2730,6.00,4.2515,5,398.0,18.70,394.92,6.78,24.10
-0.09512,0.00,12.830,0,0.4370,6.2860,45.00,4.5026,5,398.0,18.70,383.23,8.94,21.40
-0.10153,0.00,12.830,0,0.4370,6.2790,74.50,4.0522,5,398.0,18.70,373.66,11.97,20.00
-0.08707,0.00,12.830,0,0.4370,6.1400,45.80,4.0905,5,398.0,18.70,386.96,10.27,20.80
-0.05646,0.00,12.830,0,0.4370,6.2320,53.70,5.0141,5,398.0,18.70,386.40,12.34,21.20
-0.08387,0.00,12.830,0,0.4370,5.8740,36.60,4.5026,5,398.0,18.70,396.06,9.10,20.30
-0.04113,25.00,4.860,0,0.4260,6.7270,33.50,5.4007,4,281.0,19.00,396.90,5.29,28.00
-0.04462,25.00,4.860,0,0.4260,6.6190,70.40,5.4007,4,281.0,19.00,395.63,7.22,23.90
-0.03659,25.00,4.860,0,0.4260,6.3020,32.20,5.4007,4,281.0,19.00,396.90,6.72,24.80
-0.03551,25.00,4.860,0,0.4260,6.1670,46.70,5.4007,4,281.0,19.00,390.64,7.51,22.90
-0.05059,0.00,4.490,0,0.4490,6.3890,48.00,4.7794,3,247.0,18.50,396.90,9.62,23.90
-0.05735,0.00,4.490,0,0.4490,6.6300,56.10,4.4377,3,247.0,18.50,392.30,6.53,26.60
-0.05188,0.00,4.490,0,0.4490,6.0150,45.10,4.4272,3,247.0,18.50,395.99,12.86,22.50
-0.07151,0.00,4.490,0,0.4490,6.1210,56.80,3.7476,3,247.0,18.50,395.15,8.44,22.20
-0.05660,0.00,3.410,0,0.4890,7.0070,86.30,3.4217,2,270.0,17.80,396.90,5.50,23.60
-0.05302,0.00,3.410,0,0.4890,7.0790,63.10,3.4145,2,270.0,17.80,396.06,5.70,28.70
-0.04684,0.00,3.410,0,0.4890,6.4170,66.10,3.0923,2,270.0,17.80,392.18,8.81,22.60
-0.03932,0.00,3.410,0,0.4890,6.4050,73.90,3.0921,2,270.0,17.80,393.55,8.20,22.00
-0.04203,28.00,15.040,0,0.4640,6.4420,53.60,3.6659,4,270.0,18.20,395.01,8.16,22.90
-0.02875,28.00,15.040,0,0.4640,6.2110,28.90,3.6659,4,270.0,18.20,396.33,6.21,25.00
-0.04294,28.00,15.040,0,0.4640,6.2490,77.30,3.6150,4,270.0,18.20,396.90,10.59,20.60
-0.12204,0.00,2.890,0,0.4450,6.6250,57.80,3.4952,2,276.0,18.00,357.98,6.65,28.40
-0.11504,0.00,2.890,0,0.4450,6.1630,69.60,3.4952,2,276.0,18.00,391.83,11.34,21.40
-0.12083,0.00,2.890,0,0.4450,8.0690,76.00,3.4952,2,276.0,18.00,396.90,4.21,38.70
-0.08187,0.00,2.890,0,0.4450,7.8200,36.90,3.4952,2,276.0,18.00,393.53,3.57,43.80
-0.06860,0.00,2.890,0,0.4450,7.4160,62.50,3.4952,2,276.0,18.00,396.90,6.19,33.20
-0.14866,0.00,8.560,0,0.5200,6.7270,79.90,2.7778,5,384.0,20.90,394.76,9.42,27.50
-0.11432,0.00,8.560,0,0.5200,6.7810,71.30,2.8561,5,384.0,20.90,395.58,7.67,26.50
-0.22876,0.00,8.560,0,0.5200,6.4050,85.40,2.7147,5,384.0,20.90,70.80,10.63,18.60
-0.21161,0.00,8.560,0,0.5200,6.1370,87.40,2.7147,5,384.0,20.90,394.47,13.44,19.30
-0.13960,0.00,8.560,0,0.5200,6.1670,90.00,2.4210,5,384.0,20.90,392.69,12.33,20.10
-0.13262,0.00,8.560,0,0.5200,5.8510,96.70,2.1069,5,384.0,20.90,394.05,16.47,19.50
-0.17120,0.00,8.560,0,0.5200,5.8360,91.90,2.2110,5,384.0,20.90,395.67,18.66,19.50
-0.13117,0.00,8.560,0,0.5200,6.1270,85.20,2.1224,5,384.0,20.90,387.69,14.09,20.40
-0.12802,0.00,8.560,0,0.5200,6.4740,97.10,2.4329,5,384.0,20.90,395.24,12.27,19.80
-0.26363,0.00,8.560,0,0.5200,6.2290,91.20,2.5451,5,384.0,20.90,391.23,15.55,19.40
-0.10793,0.00,8.560,0,0.5200,6.1950,54.40,2.7778,5,384.0,20.90,393.49,13.00,21.70
-0.10084,0.00,10.010,0,0.5470,6.7150,81.60,2.6775,6,432.0,17.80,395.59,10.16,22.80
-0.12329,0.00,10.010,0,0.5470,5.9130,92.90,2.3534,6,432.0,17.80,394.95,16.21,18.80
-0.22212,0.00,10.010,0,0.5470,6.0920,95.40,2.5480,6,432.0,17.80,396.90,17.09,18.70
-0.14231,0.00,10.010,0,0.5470,6.2540,84.20,2.2565,6,432.0,17.80,388.74,10.45,18.50
-0.17134,0.00,10.010,0,0.5470,5.9280,88.20,2.4631,6,432.0,17.80,344.91,15.76,18.30
-0.13158,0.00,10.010,0,0.5470,6.1760,72.50,2.7301,6,432.0,17.80,393.30,12.04,21.20
-0.15098,0.00,10.010,0,0.5470,6.0210,82.60,2.7474,6,432.0,17.80,394.51,10.30,19.20
-0.13058,0.00,10.010,0,0.5470,5.8720,73.10,2.4775,6,432.0,17.80,338.63,15.37,20.40
-0.14476,0.00,10.010,0,0.5470,5.7310,65.20,2.7592,6,432.0,17.80,391.50,13.61,19.30
-0.06899,0.00,25.650,0,0.5810,5.8700,69.70,2.2577,2,188.0,19.10,389.15,14.37,22.00
-0.07165,0.00,25.650,0,0.5810,6.0040,84.10,2.1974,2,188.0,19.10,377.67,14.27,20.30
-0.09299,0.00,25.650,0,0.5810,5.9610,92.90,2.0869,2,188.0,19.10,378.09,17.93,20.50
-0.15038,0.00,25.650,0,0.5810,5.8560,97.00,1.9444,2,188.0,19.10,370.31,25.41,17.30
-0.09849,0.00,25.650,0,0.5810,5.8790,95.80,2.0063,2,188.0,19.10,379.38,17.58,18.80
-0.16902,0.00,25.650,0,0.5810,5.9860,88.40,1.9929,2,188.0,19.10,385.02,14.81,21.40
-0.38735,0.00,25.650,0,0.5810,5.6130,95.60,1.7572,2,188.0,19.10,359.29,27.26,15.70
-0.25915,0.00,21.890,0,0.6240,5.6930,96.00,1.7883,4,437.0,21.20,392.11,17.19,16.20
-0.32543,0.00,21.890,0,0.6240,6.4310,98.80,1.8125,4,437.0,21.20,396.90,15.39,18.00
-0.88125,0.00,21.890,0,0.6240,5.6370,94.70,1.9799,4,437.0,21.20,396.90,18.34,14.30
-0.34006,0.00,21.890,0,0.6240,6.4580,98.90,2.1185,4,437.0,21.20,395.04,12.60,19.20
-1.19294,0.00,21.890,0,0.6240,6.3260,97.70,2.2710,4,437.0,21.20,396.90,12.26,19.60
-0.59005,0.00,21.890,0,0.6240,6.3720,97.90,2.3274,4,437.0,21.20,385.76,11.12,23.00
-0.32982,0.00,21.890,0,0.6240,5.8220,95.40,2.4699,4,437.0,21.20,388.69,15.03,18.40
-0.97617,0.00,21.890,0,0.6240,5.7570,98.40,2.3460,4,437.0,21.20,262.76,17.31,15.60
-0.55778,0.00,21.890,0,0.6240,6.3350,98.20,2.1107,4,437.0,21.20,394.67,16.96,18.10
-0.32264,0.00,21.890,0,0.6240,5.9420,93.50,1.9669,4,437.0,21.20,378.25,16.90,17.40
-0.35233,0.00,21.890,0,0.6240,6.4540,98.40,1.8498,4,437.0,21.20,394.08,14.59,17.10
-0.24980,0.00,21.890,0,0.6240,5.8570,98.20,1.6686,4,437.0,21.20,392.04,21.32,13.30
-0.54452,0.00,21.890,0,0.6240,6.1510,97.90,1.6687,4,437.0,21.20,396.90,18.46,17.80
-0.29090,0.00,21.890,0,0.6240,6.1740,93.60,1.6119,4,437.0,21.20,388.08,24.16,14.00
-1.62864,0.00,21.890,0,0.6240,5.0190,100.00,1.4394,4,437.0,21.20,396.90,34.41,14.40
-3.32105,0.00,19.580,1,0.8710,5.4030,100.00,1.3216,5,403.0,14.70,396.90,26.82,13.40
-4.09740,0.00,19.580,0,0.8710,5.4680,100.00,1.4118,5,403.0,14.70,396.90,26.42,15.60
-2.77974,0.00,19.580,0,0.8710,4.9030,97.80,1.3459,5,403.0,14.70,396.90,29.29,11.80
-2.37934,0.00,19.580,0,0.8710,6.1300,100.00,1.4191,5,403.0,14.70,172.91,27.80,13.80
-2.15505,0.00,19.580,0,0.8710,5.6280,100.00,1.5166,5,403.0,14.70,169.27,16.65,15.60
-2.36862,0.00,19.580,0,0.8710,4.9260,95.70,1.4608,5,403.0,14.70,391.71,29.53,14.60
-2.33099,0.00,19.580,0,0.8710,5.1860,93.80,1.5296,5,403.0,14.70,356.99,28.32,17.80
-2.73397,0.00,19.580,0,0.8710,5.5970,94.90,1.5257,5,403.0,14.70,351.85,21.45,15.40
-1.65660,0.00,19.580,0,0.8710,6.1220,97.30,1.6180,5,403.0,14.70,372.80,14.10,21.50
-1.49632,0.00,19.580,0,0.8710,5.4040,100.00,1.5916,5,403.0,14.70,341.60,13.28,19.60
-1.12658,0.00,19.580,1,0.8710,5.0120,88.00,1.6102,5,403.0,14.70,343.28,12.12,15.30
-2.14918,0.00,19.580,0,0.8710,5.7090,98.50,1.6232,5,403.0,14.70,261.95,15.79,19.40
-1.41385,0.00,19.580,1,0.8710,6.1290,96.00,1.7494,5,403.0,14.70,321.02,15.12,17.00
-3.53501,0.00,19.580,1,0.8710,6.1520,82.60,1.7455,5,403.0,14.70,88.01,15.02,15.60
-2.44668,0.00,19.580,0,0.8710,5.2720,94.00,1.7364,5,403.0,14.70,88.63,16.14,13.10
-1.22358,0.00,19.580,0,0.6050,6.9430,97.40,1.8773,5,403.0,14.70,363.43,4.59,41.30
-1.34284,0.00,19.580,0,0.6050,6.0660,100.00,1.7573,5,403.0,14.70,353.89,6.43,24.30
-1.42502,0.00,19.580,0,0.8710,6.5100,100.00,1.7659,5,403.0,14.70,364.31,7.39,23.30
-1.27346,0.00,19.580,1,0.6050,6.2500,92.60,1.7984,5,403.0,14.70,338.92,5.50,27.00
-1.46336,0.00,19.580,0,0.6050,7.4890,90.80,1.9709,5,403.0,14.70,374.43,1.73,50.00
-1.83377,0.00,19.580,1,0.6050,7.8020,98.20,2.0407,5,403.0,14.70,389.61,1.92,50.00
-1.51902,0.00,19.580,1,0.6050,8.3750,93.90,2.1620,5,403.0,14.70,388.45,3.32,50.00
-2.24236,0.00,19.580,0,0.6050,5.8540,91.80,2.4220,5,403.0,14.70,395.11,11.64,22.70
-2.92400,0.00,19.580,0,0.6050,6.1010,93.00,2.2834,5,403.0,14.70,240.16,9.81,25.00
-2.01019,0.00,19.580,0,0.6050,7.9290,96.20,2.0459,5,403.0,14.70,369.30,3.70,50.00
-1.80028,0.00,19.580,0,0.6050,5.8770,79.20,2.4259,5,403.0,14.70,227.61,12.14,23.80
-2.30040,0.00,19.580,0,0.6050,6.3190,96.10,2.1000,5,403.0,14.70,297.09,11.10,23.80
-2.44953,0.00,19.580,0,0.6050,6.4020,95.20,2.2625,5,403.0,14.70,330.04,11.32,22.30
-1.20742,0.00,19.580,0,0.6050,5.8750,94.60,2.4259,5,403.0,14.70,292.29,14.43,17.40
-2.31390,0.00,19.580,0,0.6050,5.8800,97.30,2.3887,5,403.0,14.70,348.13,12.03,19.10
-0.13914,0.00,4.050,0,0.5100,5.5720,88.50,2.5961,5,296.0,16.60,396.90,14.69,23.10
-0.09178,0.00,4.050,0,0.5100,6.4160,84.10,2.6463,5,296.0,16.60,395.50,9.04,23.60
-0.08447,0.00,4.050,0,0.5100,5.8590,68.70,2.7019,5,296.0,16.60,393.23,9.64,22.60
-0.06664,0.00,4.050,0,0.5100,6.5460,33.10,3.1323,5,296.0,16.60,390.96,5.33,29.40
-0.07022,0.00,4.050,0,0.5100,6.0200,47.20,3.5549,5,296.0,16.60,393.23,10.11,23.20
-0.05425,0.00,4.050,0,0.5100,6.3150,73.40,3.3175,5,296.0,16.60,395.60,6.29,24.60
-0.06642,0.00,4.050,0,0.5100,6.8600,74.40,2.9153,5,296.0,16.60,391.27,6.92,29.90
-0.05780,0.00,2.460,0,0.4880,6.9800,58.40,2.8290,3,193.0,17.80,396.90,5.04,37.20
-0.06588,0.00,2.460,0,0.4880,7.7650,83.30,2.7410,3,193.0,17.80,395.56,7.56,39.80
-0.06888,0.00,2.460,0,0.4880,6.1440,62.20,2.5979,3,193.0,17.80,396.90,9.45,36.20
-0.09103,0.00,2.460,0,0.4880,7.1550,92.20,2.7006,3,193.0,17.80,394.12,4.82,37.90
-0.10008,0.00,2.460,0,0.4880,6.5630,95.60,2.8470,3,193.0,17.80,396.90,5.68,32.50
-0.08308,0.00,2.460,0,0.4880,5.6040,89.80,2.9879,3,193.0,17.80,391.00,13.98,26.40
-0.06047,0.00,2.460,0,0.4880,6.1530,68.80,3.2797,3,193.0,17.80,387.11,13.15,29.60
-0.05602,0.00,2.460,0,0.4880,7.8310,53.60,3.1992,3,193.0,17.80,392.63,4.45,50.00
-0.07875,45.00,3.440,0,0.4370,6.7820,41.10,3.7886,5,398.0,15.20,393.87,6.68,32.00
-0.12579,45.00,3.440,0,0.4370,6.5560,29.10,4.5667,5,398.0,15.20,382.84,4.56,29.80
-0.08370,45.00,3.440,0,0.4370,7.1850,38.90,4.5667,5,398.0,15.20,396.90,5.39,34.90
-0.09068,45.00,3.440,0,0.4370,6.9510,21.50,6.4798,5,398.0,15.20,377.68,5.10,37.00
-0.06911,45.00,3.440,0,0.4370,6.7390,30.80,6.4798,5,398.0,15.20,389.71,4.69,30.50
-0.08664,45.00,3.440,0,0.4370,7.1780,26.30,6.4798,5,398.0,15.20,390.49,2.87,36.40
-0.02187,60.00,2.930,0,0.4010,6.8000,9.90,6.2196,1,265.0,15.60,393.37,5.03,31.10
-0.01439,60.00,2.930,0,0.4010,6.6040,18.80,6.2196,1,265.0,15.60,376.70,4.38,29.10
-0.01381,80.00,0.460,0,0.4220,7.8750,32.00,5.6484,4,255.0,14.40,394.23,2.97,50.00
-0.04011,80.00,1.520,0,0.4040,7.2870,34.10,7.3090,2,329.0,12.60,396.90,4.08,33.30
-0.04666,80.00,1.520,0,0.4040,7.1070,36.60,7.3090,2,329.0,12.60,354.31,8.61,30.30
-0.03768,80.00,1.520,0,0.4040,7.2740,38.30,7.3090,2,329.0,12.60,392.20,6.62,34.60
-0.03150,95.00,1.470,0,0.4030,6.9750,15.30,7.6534,3,402.0,17.00,396.90,4.56,34.90
-0.01778,95.00,1.470,0,0.4030,7.1350,13.90,7.6534,3,402.0,17.00,384.30,4.45,32.90
-0.03445,82.50,2.030,0,0.4150,6.1620,38.40,6.2700,2,348.0,14.70,393.77,7.43,24.10
-0.02177,82.50,2.030,0,0.4150,7.6100,15.70,6.2700,2,348.0,14.70,395.38,3.11,42.30
-0.03510,95.00,2.680,0,0.4161,7.8530,33.20,5.1180,4,224.0,14.70,392.78,3.81,48.50
-0.02009,95.00,2.680,0,0.4161,8.0340,31.90,5.1180,4,224.0,14.70,390.55,2.88,50.00
-0.13642,0.00,10.590,0,0.4890,5.8910,22.30,3.9454,4,277.0,18.60,396.90,10.87,22.60
-0.22969,0.00,10.590,0,0.4890,6.3260,52.50,4.3549,4,277.0,18.60,394.87,10.97,24.40
-0.25199,0.00,10.590,0,0.4890,5.7830,72.70,4.3549,4,277.0,18.60,389.43,18.06,22.50
-0.13587,0.00,10.590,1,0.4890,6.0640,59.10,4.2392,4,277.0,18.60,381.32,14.66,24.40
-0.43571,0.00,10.590,1,0.4890,5.3440,100.00,3.8750,4,277.0,18.60,396.90,23.09,20.00
-0.17446,0.00,10.590,1,0.4890,5.9600,92.10,3.8771,4,277.0,18.60,393.25,17.27,21.70
-0.37578,0.00,10.590,1,0.4890,5.4040,88.60,3.6650,4,277.0,18.60,395.24,23.98,19.30
-0.21719,0.00,10.590,1,0.4890,5.8070,53.80,3.6526,4,277.0,18.60,390.94,16.03,22.40
-0.14052,0.00,10.590,0,0.4890,6.3750,32.30,3.9454,4,277.0,18.60,385.81,9.38,28.10
-0.28955,0.00,10.590,0,0.4890,5.4120,9.80,3.5875,4,277.0,18.60,348.93,29.55,23.70
-0.19802,0.00,10.590,0,0.4890,6.1820,42.40,3.9454,4,277.0,18.60,393.63,9.47,25.00
-0.04560,0.00,13.890,1,0.5500,5.8880,56.00,3.1121,5,276.0,16.40,392.80,13.51,23.30
-0.07013,0.00,13.890,0,0.5500,6.6420,85.10,3.4211,5,276.0,16.40,392.78,9.69,28.70
-0.11069,0.00,13.890,1,0.5500,5.9510,93.80,2.8893,5,276.0,16.40,396.90,17.92,21.50
-0.11425,0.00,13.890,1,0.5500,6.3730,92.40,3.3633,5,276.0,16.40,393.74,10.50,23.00
-0.35809,0.00,6.200,1,0.5070,6.9510,88.50,2.8617,8,307.0,17.40,391.70,9.71,26.70
-0.40771,0.00,6.200,1,0.5070,6.1640,91.30,3.0480,8,307.0,17.40,395.24,21.46,21.70
-0.62356,0.00,6.200,1,0.5070,6.8790,77.70,3.2721,8,307.0,17.40,390.39,9.93,27.50
-0.61470,0.00,6.200,0,0.5070,6.6180,80.80,3.2721,8,307.0,17.40,396.90,7.60,30.10
-0.31533,0.00,6.200,0,0.5040,8.2660,78.30,2.8944,8,307.0,17.40,385.05,4.14,44.80
-0.52693,0.00,6.200,0,0.5040,8.7250,83.00,2.8944,8,307.0,17.40,382.00,4.63,50.00
-0.38214,0.00,6.200,0,0.5040,8.0400,86.50,3.2157,8,307.0,17.40,387.38,3.13,37.60
-0.41238,0.00,6.200,0,0.5040,7.1630,79.90,3.2157,8,307.0,17.40,372.08,6.36,31.60
-0.29819,0.00,6.200,0,0.5040,7.6860,17.00,3.3751,8,307.0,17.40,377.51,3.92,46.70
-0.44178,0.00,6.200,0,0.5040,6.5520,21.40,3.3751,8,307.0,17.40,380.34,3.76,31.50
-0.53700,0.00,6.200,0,0.5040,5.9810,68.10,3.6715,8,307.0,17.40,378.35,11.65,24.30
-0.46296,0.00,6.200,0,0.5040,7.4120,76.90,3.6715,8,307.0,17.40,376.14,5.25,31.70
-0.57529,0.00,6.200,0,0.5070,8.3370,73.30,3.8384,8,307.0,17.40,385.91,2.47,41.70
-0.33147,0.00,6.200,0,0.5070,8.2470,70.40,3.6519,8,307.0,17.40,378.95,3.95,48.30
-0.44791,0.00,6.200,1,0.5070,6.7260,66.50,3.6519,8,307.0,17.40,360.20,8.05,29.00
-0.33045,0.00,6.200,0,0.5070,6.0860,61.50,3.6519,8,307.0,17.40,376.75,10.88,24.00
-0.52058,0.00,6.200,1,0.5070,6.6310,76.50,4.1480,8,307.0,17.40,388.45,9.54,25.10
-0.51183,0.00,6.200,0,0.5070,7.3580,71.60,4.1480,8,307.0,17.40,390.07,4.73,31.50
-0.08244,30.00,4.930,0,0.4280,6.4810,18.50,6.1899,6,300.0,16.60,379.41,6.36,23.70
-0.09252,30.00,4.930,0,0.4280,6.6060,42.20,6.1899,6,300.0,16.60,383.78,7.37,23.30
-0.11329,30.00,4.930,0,0.4280,6.8970,54.30,6.3361,6,300.0,16.60,391.25,11.38,22.00
-0.10612,30.00,4.930,0,0.4280,6.0950,65.10,6.3361,6,300.0,16.60,394.62,12.40,20.10
-0.10290,30.00,4.930,0,0.4280,6.3580,52.90,7.0355,6,300.0,16.60,372.75,11.22,22.20
-0.12757,30.00,4.930,0,0.4280,6.3930,7.80,7.0355,6,300.0,16.60,374.71,5.19,23.70
-0.20608,22.00,5.860,0,0.4310,5.5930,76.50,7.9549,7,330.0,19.10,372.49,12.50,17.60
-0.19133,22.00,5.860,0,0.4310,5.6050,70.20,7.9549,7,330.0,19.10,389.13,18.46,18.50
-0.33983,22.00,5.860,0,0.4310,6.1080,34.90,8.0555,7,330.0,19.10,390.18,9.16,24.30
-0.19657,22.00,5.860,0,0.4310,6.2260,79.20,8.0555,7,330.0,19.10,376.14,10.15,20.50
-0.16439,22.00,5.860,0,0.4310,6.4330,49.10,7.8265,7,330.0,19.10,374.71,9.52,24.50
-0.19073,22.00,5.860,0,0.4310,6.7180,17.50,7.8265,7,330.0,19.10,393.74,6.56,26.20
-0.14030,22.00,5.860,0,0.4310,6.4870,13.00,7.3967,7,330.0,19.10,396.28,5.90,24.40
-0.21409,22.00,5.860,0,0.4310,6.4380,8.90,7.3967,7,330.0,19.10,377.07,3.59,24.80
-0.08221,22.00,5.860,0,0.4310,6.9570,6.80,8.9067,7,330.0,19.10,386.09,3.53,29.60
-0.36894,22.00,5.860,0,0.4310,8.2590,8.40,8.9067,7,330.0,19.10,396.90,3.54,42.80
-0.04819,80.00,3.640,0,0.3920,6.1080,32.00,9.2203,1,315.0,16.40,392.89,6.57,21.90
-0.03548,80.00,3.640,0,0.3920,5.8760,19.10,9.2203,1,315.0,16.40,395.18,9.25,20.90
-0.01538,90.00,3.750,0,0.3940,7.4540,34.20,6.3361,3,244.0,15.90,386.34,3.11,44.00
-0.61154,20.00,3.970,0,0.6470,8.7040,86.90,1.8010,5,264.0,13.00,389.70,5.12,50.00
-0.66351,20.00,3.970,0,0.6470,7.3330,100.00,1.8946,5,264.0,13.00,383.29,7.79,36.00
-0.65665,20.00,3.970,0,0.6470,6.8420,100.00,2.0107,5,264.0,13.00,391.93,6.90,30.10
-0.54011,20.00,3.970,0,0.6470,7.2030,81.80,2.1121,5,264.0,13.00,392.80,9.59,33.80
-0.53412,20.00,3.970,0,0.6470,7.5200,89.40,2.1398,5,264.0,13.00,388.37,7.26,43.10
-0.52014,20.00,3.970,0,0.6470,8.3980,91.50,2.2885,5,264.0,13.00,386.86,5.91,48.80
-0.82526,20.00,3.970,0,0.6470,7.3270,94.50,2.0788,5,264.0,13.00,393.42,11.25,31.00
-0.55007,20.00,3.970,0,0.6470,7.2060,91.60,1.9301,5,264.0,13.00,387.89,8.10,36.50
-0.76162,20.00,3.970,0,0.6470,5.5600,62.80,1.9865,5,264.0,13.00,392.40,10.45,22.80
-0.78570,20.00,3.970,0,0.6470,7.0140,84.60,2.1329,5,264.0,13.00,384.07,14.79,30.70
-0.57834,20.00,3.970,0,0.5750,8.2970,67.00,2.4216,5,264.0,13.00,384.54,7.44,50.00
-0.54050,20.00,3.970,0,0.5750,7.4700,52.60,2.8720,5,264.0,13.00,390.30,3.16,43.50
-0.09065,20.00,6.960,1,0.4640,5.9200,61.50,3.9175,3,223.0,18.60,391.34,13.65,20.70
-0.29916,20.00,6.960,0,0.4640,5.8560,42.10,4.4290,3,223.0,18.60,388.65,13.00,21.10
-0.16211,20.00,6.960,0,0.4640,6.2400,16.30,4.4290,3,223.0,18.60,396.90,6.59,25.20
-0.11460,20.00,6.960,0,0.4640,6.5380,58.70,3.9175,3,223.0,18.60,394.96,7.73,24.40
-0.22188,20.00,6.960,1,0.4640,7.6910,51.80,4.3665,3,223.0,18.60,390.77,6.58,35.20
-0.05644,40.00,6.410,1,0.4470,6.7580,32.90,4.0776,4,254.0,17.60,396.90,3.53,32.40
-0.09604,40.00,6.410,0,0.4470,6.8540,42.80,4.2673,4,254.0,17.60,396.90,2.98,32.00
-0.10469,40.00,6.410,1,0.4470,7.2670,49.00,4.7872,4,254.0,17.60,389.25,6.05,33.20
-0.06127,40.00,6.410,1,0.4470,6.8260,27.60,4.8628,4,254.0,17.60,393.45,4.16,33.10
-0.07978,40.00,6.410,0,0.4470,6.4820,32.10,4.1403,4,254.0,17.60,396.90,7.19,29.10
-0.21038,20.00,3.330,0,0.4429,6.8120,32.20,4.1007,5,216.0,14.90,396.90,4.85,35.10
-0.03578,20.00,3.330,0,0.4429,7.8200,64.50,4.6947,5,216.0,14.90,387.31,3.76,45.40
-0.03705,20.00,3.330,0,0.4429,6.9680,37.20,5.2447,5,216.0,14.90,392.23,4.59,35.40
-0.06129,20.00,3.330,1,0.4429,7.6450,49.70,5.2119,5,216.0,14.90,377.07,3.01,46.00
-0.01501,90.00,1.210,1,0.4010,7.9230,24.80,5.8850,1,198.0,13.60,395.52,3.16,50.00
-0.00906,90.00,2.970,0,0.4000,7.0880,20.80,7.3073,1,285.0,15.30,394.72,7.85,32.20
-0.01096,55.00,2.250,0,0.3890,6.4530,31.90,7.3073,1,300.0,15.30,394.72,8.23,22.00
-0.01965,80.00,1.760,0,0.3850,6.2300,31.50,9.0892,1,241.0,18.20,341.60,12.93,20.10
-0.03871,52.50,5.320,0,0.4050,6.2090,31.30,7.3172,6,293.0,16.60,396.90,7.14,23.20
-0.04590,52.50,5.320,0,0.4050,6.3150,45.60,7.3172,6,293.0,16.60,396.90,7.60,22.30
-0.04297,52.50,5.320,0,0.4050,6.5650,22.90,7.3172,6,293.0,16.60,371.72,9.51,24.80
-0.03502,80.00,4.950,0,0.4110,6.8610,27.90,5.1167,4,245.0,19.20,396.90,3.33,28.50
-0.07886,80.00,4.950,0,0.4110,7.1480,27.70,5.1167,4,245.0,19.20,396.90,3.56,37.30
-0.03615,80.00,4.950,0,0.4110,6.6300,23.40,5.1167,4,245.0,19.20,396.90,4.70,27.90
-0.08265,0.00,13.920,0,0.4370,6.1270,18.40,5.5027,4,289.0,16.00,396.90,8.58,23.90
-0.08199,0.00,13.920,0,0.4370,6.0090,42.30,5.5027,4,289.0,16.00,396.90,10.40,21.70
-0.12932,0.00,13.920,0,0.4370,6.6780,31.10,5.9604,4,289.0,16.00,396.90,6.27,28.60
-0.05372,0.00,13.920,0,0.4370,6.5490,51.00,5.9604,4,289.0,16.00,392.85,7.39,27.10
-0.14103,0.00,13.920,0,0.4370,5.7900,58.00,6.3200,4,289.0,16.00,396.90,15.84,20.30
-0.06466,70.00,2.240,0,0.4000,6.3450,20.10,7.8278,5,358.0,14.80,368.24,4.97,22.50
-0.05561,70.00,2.240,0,0.4000,7.0410,10.00,7.8278,5,358.0,14.80,371.58,4.74,29.00
-0.04417,70.00,2.240,0,0.4000,6.8710,47.40,7.8278,5,358.0,14.80,390.86,6.07,24.80
-0.03537,34.00,6.090,0,0.4330,6.5900,40.40,5.4917,7,329.0,16.10,395.75,9.50,22.00
-0.09266,34.00,6.090,0,0.4330,6.4950,18.40,5.4917,7,329.0,16.10,383.61,8.67,26.40
-0.10000,34.00,6.090,0,0.4330,6.9820,17.70,5.4917,7,329.0,16.10,390.43,4.86,33.10
-0.05515,33.00,2.180,0,0.4720,7.2360,41.10,4.0220,7,222.0,18.40,393.68,6.93,36.10
-0.05479,33.00,2.180,0,0.4720,6.6160,58.10,3.3700,7,222.0,18.40,393.36,8.93,28.40
-0.07503,33.00,2.180,0,0.4720,7.4200,71.90,3.0992,7,222.0,18.40,396.90,6.47,33.40
-0.04932,33.00,2.180,0,0.4720,6.8490,70.30,3.1827,7,222.0,18.40,396.90,7.53,28.20
-0.49298,0.00,9.900,0,0.5440,6.6350,82.50,3.3175,4,304.0,18.40,396.90,4.54,22.80
-0.34940,0.00,9.900,0,0.5440,5.9720,76.70,3.1025,4,304.0,18.40,396.24,9.97,20.30
-2.63548,0.00,9.900,0,0.5440,4.9730,37.80,2.5194,4,304.0,18.40,350.45,12.64,16.10
-0.79041,0.00,9.900,0,0.5440,6.1220,52.80,2.6403,4,304.0,18.40,396.90,5.98,22.10
-0.26169,0.00,9.900,0,0.5440,6.0230,90.40,2.8340,4,304.0,18.40,396.30,11.72,19.40
-0.26938,0.00,9.900,0,0.5440,6.2660,82.80,3.2628,4,304.0,18.40,393.39,7.90,21.60
-0.36920,0.00,9.900,0,0.5440,6.5670,87.30,3.6023,4,304.0,18.40,395.69,9.28,23.80
-0.25356,0.00,9.900,0,0.5440,5.7050,77.70,3.9450,4,304.0,18.40,396.42,11.50,16.20
-0.31827,0.00,9.900,0,0.5440,5.9140,83.20,3.9986,4,304.0,18.40,390.70,18.33,17.80
-0.24522,0.00,9.900,0,0.5440,5.7820,71.70,4.0317,4,304.0,18.40,396.90,15.94,19.80
-0.40202,0.00,9.900,0,0.5440,6.3820,67.20,3.5325,4,304.0,18.40,395.21,10.36,23.10
-0.47547,0.00,9.900,0,0.5440,6.1130,58.80,4.0019,4,304.0,18.40,396.23,12.73,21.00
-0.16760,0.00,7.380,0,0.4930,6.4260,52.30,4.5404,5,287.0,19.60,396.90,7.20,23.80
-0.18159,0.00,7.380,0,0.4930,6.3760,54.30,4.5404,5,287.0,19.60,396.90,6.87,23.10
-0.35114,0.00,7.380,0,0.4930,6.0410,49.90,4.7211,5,287.0,19.60,396.90,7.70,20.40
-0.28392,0.00,7.380,0,0.4930,5.7080,74.30,4.7211,5,287.0,19.60,391.13,11.74,18.50
-0.34109,0.00,7.380,0,0.4930,6.4150,40.10,4.7211,5,287.0,19.60,396.90,6.12,25.00
-0.19186,0.00,7.380,0,0.4930,6.4310,14.70,5.4159,5,287.0,19.60,393.68,5.08,24.60
-0.30347,0.00,7.380,0,0.4930,6.3120,28.90,5.4159,5,287.0,19.60,396.90,6.15,23.00
-0.24103,0.00,7.380,0,0.4930,6.0830,43.70,5.4159,5,287.0,19.60,396.90,12.79,22.20
-0.06617,0.00,3.240,0,0.4600,5.8680,25.80,5.2146,4,430.0,16.90,382.44,9.97,19.30
-0.06724,0.00,3.240,0,0.4600,6.3330,17.20,5.2146,4,430.0,16.90,375.21,7.34,22.60
-0.04544,0.00,3.240,0,0.4600,6.1440,32.20,5.8736,4,430.0,16.90,368.57,9.09,19.80
-0.05023,35.00,6.060,0,0.4379,5.7060,28.40,6.6407,1,304.0,16.90,394.02,12.43,17.10
-0.03466,35.00,6.060,0,0.4379,6.0310,23.30,6.6407,1,304.0,16.90,362.25,7.83,19.40
-0.05083,0.00,5.190,0,0.5150,6.3160,38.10,6.4584,5,224.0,20.20,389.71,5.68,22.20
-0.03738,0.00,5.190,0,0.5150,6.3100,38.50,6.4584,5,224.0,20.20,389.40,6.75,20.70
-0.03961,0.00,5.190,0,0.5150,6.0370,34.50,5.9853,5,224.0,20.20,396.90,8.01,21.10
-0.03427,0.00,5.190,0,0.5150,5.8690,46.30,5.2311,5,224.0,20.20,396.90,9.80,19.50
-0.03041,0.00,5.190,0,0.5150,5.8950,59.60,5.6150,5,224.0,20.20,394.81,10.56,18.50
-0.03306,0.00,5.190,0,0.5150,6.0590,37.30,4.8122,5,224.0,20.20,396.14,8.51,20.60
-0.05497,0.00,5.190,0,0.5150,5.9850,45.40,4.8122,5,224.0,20.20,396.90,9.74,19.00
-0.06151,0.00,5.190,0,0.5150,5.9680,58.50,4.8122,5,224.0,20.20,396.90,9.29,18.70
-0.01301,35.00,1.520,0,0.4420,7.2410,49.30,7.0379,1,284.0,15.50,394.74,5.49,32.70
-0.02498,0.00,1.890,0,0.5180,6.5400,59.70,6.2669,1,422.0,15.90,389.96,8.65,16.50
-0.02543,55.00,3.780,0,0.4840,6.6960,56.40,5.7321,5,370.0,17.60,396.90,7.18,23.90
-0.03049,55.00,3.780,0,0.4840,6.8740,28.10,6.4654,5,370.0,17.60,387.97,4.61,31.20
-0.03113,0.00,4.390,0,0.4420,6.0140,48.50,8.0136,3,352.0,18.80,385.64,10.53,17.50
-0.06162,0.00,4.390,0,0.4420,5.8980,52.30,8.0136,3,352.0,18.80,364.61,12.67,17.20
-0.01870,85.00,4.150,0,0.4290,6.5160,27.70,8.5353,4,351.0,17.90,392.43,6.36,23.10
-0.01501,80.00,2.010,0,0.4350,6.6350,29.70,8.3440,4,280.0,17.00,390.94,5.99,24.50
-0.02899,40.00,1.250,0,0.4290,6.9390,34.50,8.7921,1,335.0,19.70,389.85,5.89,26.60
-0.06211,40.00,1.250,0,0.4290,6.4900,44.40,8.7921,1,335.0,19.70,396.90,5.98,22.90
-0.07950,60.00,1.690,0,0.4110,6.5790,35.90,10.7103,4,411.0,18.30,370.78,5.49,24.10
-0.07244,60.00,1.690,0,0.4110,5.8840,18.50,10.7103,4,411.0,18.30,392.33,7.79,18.60
-0.01709,90.00,2.020,0,0.4100,6.7280,36.10,12.1265,5,187.0,17.00,384.46,4.50,30.10
-0.04301,80.00,1.910,0,0.4130,5.6630,21.90,10.5857,4,334.0,22.00,382.80,8.05,18.20
-0.10659,80.00,1.910,0,0.4130,5.9360,19.50,10.5857,4,334.0,22.00,376.04,5.57,20.60
-8.98296,0.00,18.100,1,0.7700,6.2120,97.40,2.1222,24,666.0,20.20,377.73,17.60,17.80
-3.84970,0.00,18.100,1,0.7700,6.3950,91.00,2.5052,24,666.0,20.20,391.34,13.27,21.70
-5.20177,0.00,18.100,1,0.7700,6.1270,83.40,2.7227,24,666.0,20.20,395.43,11.48,22.70
-4.26131,0.00,18.100,0,0.7700,6.1120,81.30,2.5091,24,666.0,20.20,390.74,12.67,22.60
-4.54192,0.00,18.100,0,0.7700,6.3980,88.00,2.5182,24,666.0,20.20,374.56,7.79,25.00
-3.83684,0.00,18.100,0,0.7700,6.2510,91.10,2.2955,24,666.0,20.20,350.65,14.19,19.90
-3.67822,0.00,18.100,0,0.7700,5.3620,96.20,2.1036,24,666.0,20.20,380.79,10.19,20.80
-4.22239,0.00,18.100,1,0.7700,5.8030,89.00,1.9047,24,666.0,20.20,353.04,14.64,16.80
-3.47428,0.00,18.100,1,0.7180,8.7800,82.90,1.9047,24,666.0,20.20,354.55,5.29,21.90
-4.55587,0.00,18.100,0,0.7180,3.5610,87.90,1.6132,24,666.0,20.20,354.70,7.12,27.50
-3.69695,0.00,18.100,0,0.7180,4.9630,91.40,1.7523,24,666.0,20.20,316.03,14.00,21.90
-13.52220,0.00,18.100,0,0.6310,3.8630,100.00,1.5106,24,666.0,20.20,131.42,13.33,23.10
-4.89822,0.00,18.100,0,0.6310,4.9700,100.00,1.3325,24,666.0,20.20,375.52,3.26,50.00
-5.66998,0.00,18.100,1,0.6310,6.6830,96.80,1.3567,24,666.0,20.20,375.33,3.73,50.00
-6.53876,0.00,18.100,1,0.6310,7.0160,97.50,1.2024,24,666.0,20.20,392.05,2.96,50.00
-9.23230,0.00,18.100,0,0.6310,6.2160,100.00,1.1691,24,666.0,20.20,366.15,9.53,50.00
-8.26725,0.00,18.100,1,0.6680,5.8750,89.60,1.1296,24,666.0,20.20,347.88,8.88,50.00
-11.10810,0.00,18.100,0,0.6680,4.9060,100.00,1.1742,24,666.0,20.20,396.90,34.77,13.80
-18.49820,0.00,18.100,0,0.6680,4.1380,100.00,1.1370,24,666.0,20.20,396.90,37.97,13.80
-19.60910,0.00,18.100,0,0.6710,7.3130,97.90,1.3163,24,666.0,20.20,396.90,13.44,15.00
-15.28800,0.00,18.100,0,0.6710,6.6490,93.30,1.3449,24,666.0,20.20,363.02,23.24,13.90
-9.82349,0.00,18.100,0,0.6710,6.7940,98.80,1.3580,24,666.0,20.20,396.90,21.24,13.30
-23.64820,0.00,18.100,0,0.6710,6.3800,96.20,1.3861,24,666.0,20.20,396.90,23.69,13.10
-17.86670,0.00,18.100,0,0.6710,6.2230,100.00,1.3861,24,666.0,20.20,393.74,21.78,10.20
-88.97620,0.00,18.100,0,0.6710,6.9680,91.90,1.4165,24,666.0,20.20,396.90,17.21,10.40
-15.87440,0.00,18.100,0,0.6710,6.5450,99.10,1.5192,24,666.0,20.20,396.90,21.08,10.90
-9.18702,0.00,18.100,0,0.7000,5.5360,100.00,1.5804,24,666.0,20.20,396.90,23.60,11.30
-7.99248,0.00,18.100,0,0.7000,5.5200,100.00,1.5331,24,666.0,20.20,396.90,24.56,12.30
-20.08490,0.00,18.100,0,0.7000,4.3680,91.20,1.4395,24,666.0,20.20,285.83,30.63,8.80
-16.81180,0.00,18.100,0,0.7000,5.2770,98.10,1.4261,24,666.0,20.20,396.90,30.81,7.20
-24.39380,0.00,18.100,0,0.7000,4.6520,100.00,1.4672,24,666.0,20.20,396.90,28.28,10.50
-22.59710,0.00,18.100,0,0.7000,5.0000,89.50,1.5184,24,666.0,20.20,396.90,31.99,7.40
-14.33370,0.00,18.100,0,0.7000,4.8800,100.00,1.5895,24,666.0,20.20,372.92,30.62,10.20
-8.15174,0.00,18.100,0,0.7000,5.3900,98.90,1.7281,24,666.0,20.20,396.90,20.85,11.50
-6.96215,0.00,18.100,0,0.7000,5.7130,97.00,1.9265,24,666.0,20.20,394.43,17.11,15.10
-5.29305,0.00,18.100,0,0.7000,6.0510,82.50,2.1678,24,666.0,20.20,378.38,18.76,23.20
-11.57790,0.00,18.100,0,0.7000,5.0360,97.00,1.7700,24,666.0,20.20,396.90,25.68,9.70
-8.64476,0.00,18.100,0,0.6930,6.1930,92.60,1.7912,24,666.0,20.20,396.90,15.17,13.80
-13.35980,0.00,18.100,0,0.6930,5.8870,94.70,1.7821,24,666.0,20.20,396.90,16.35,12.70
-8.71675,0.00,18.100,0,0.6930,6.4710,98.80,1.7257,24,666.0,20.20,391.98,17.12,13.10
-5.87205,0.00,18.100,0,0.6930,6.4050,96.00,1.6768,24,666.0,20.20,396.90,19.37,12.50
-7.67202,0.00,18.100,0,0.6930,5.7470,98.90,1.6334,24,666.0,20.20,393.10,19.92,8.50
-38.35180,0.00,18.100,0,0.6930,5.4530,100.00,1.4896,24,666.0,20.20,396.90,30.59,5.00
-9.91655,0.00,18.100,0,0.6930,5.8520,77.80,1.5004,24,666.0,20.20,338.16,29.97,6.30
-25.04610,0.00,18.100,0,0.6930,5.9870,100.00,1.5888,24,666.0,20.20,396.90,26.77,5.60
-14.23620,0.00,18.100,0,0.6930,6.3430,100.00,1.5741,24,666.0,20.20,396.90,20.32,7.20
-9.59571,0.00,18.100,0,0.6930,6.4040,100.00,1.6390,24,666.0,20.20,376.11,20.31,12.10
-24.80170,0.00,18.100,0,0.6930,5.3490,96.00,1.7028,24,666.0,20.20,396.90,19.77,8.30
-41.52920,0.00,18.100,0,0.6930,5.5310,85.40,1.6074,24,666.0,20.20,329.46,27.38,8.50
-67.92080,0.00,18.100,0,0.6930,5.6830,100.00,1.4254,24,666.0,20.20,384.97,22.98,5.00
-20.71620,0.00,18.100,0,0.6590,4.1380,100.00,1.1781,24,666.0,20.20,370.22,23.34,11.90
-11.95110,0.00,18.100,0,0.6590,5.6080,100.00,1.2852,24,666.0,20.20,332.09,12.13,27.90
-7.40389,0.00,18.100,0,0.5970,5.6170,97.90,1.4547,24,666.0,20.20,314.64,26.40,17.20
-14.43830,0.00,18.100,0,0.5970,6.8520,100.00,1.4655,24,666.0,20.20,179.36,19.78,27.50
-51.13580,0.00,18.100,0,0.5970,5.7570,100.00,1.4130,24,666.0,20.20,2.60,10.11,15.00
-14.05070,0.00,18.100,0,0.5970,6.6570,100.00,1.5275,24,666.0,20.20,35.05,21.22,17.20
-18.81100,0.00,18.100,0,0.5970,4.6280,100.00,1.5539,24,666.0,20.20,28.79,34.37,17.90
-28.65580,0.00,18.100,0,0.5970,5.1550,100.00,1.5894,24,666.0,20.20,210.97,20.08,16.30
-45.74610,0.00,18.100,0,0.6930,4.5190,100.00,1.6582,24,666.0,20.20,88.27,36.98,7.00
-18.08460,0.00,18.100,0,0.6790,6.4340,100.00,1.8347,24,666.0,20.20,27.25,29.05,7.20
-10.83420,0.00,18.100,0,0.6790,6.7820,90.80,1.8195,24,666.0,20.20,21.57,25.79,7.50
-25.94060,0.00,18.100,0,0.6790,5.3040,89.10,1.6475,24,666.0,20.20,127.36,26.64,10.40
-73.53410,0.00,18.100,0,0.6790,5.9570,100.00,1.8026,24,666.0,20.20,16.45,20.62,8.80
-11.81230,0.00,18.100,0,0.7180,6.8240,76.50,1.7940,24,666.0,20.20,48.45,22.74,8.40
-11.08740,0.00,18.100,0,0.7180,6.4110,100.00,1.8589,24,666.0,20.20,318.75,15.02,16.70
-7.02259,0.00,18.100,0,0.7180,6.0060,95.30,1.8746,24,666.0,20.20,319.98,15.70,14.20
-12.04820,0.00,18.100,0,0.6140,5.6480,87.60,1.9512,24,666.0,20.20,291.55,14.10,20.80
-7.05042,0.00,18.100,0,0.6140,6.1030,85.10,2.0218,24,666.0,20.20,2.52,23.29,13.40
-8.79212,0.00,18.100,0,0.5840,5.5650,70.60,2.0635,24,666.0,20.20,3.65,17.16,11.70
-15.86030,0.00,18.100,0,0.6790,5.8960,95.40,1.9096,24,666.0,20.20,7.68,24.39,8.30
-12.24720,0.00,18.100,0,0.5840,5.8370,59.70,1.9976,24,666.0,20.20,24.65,15.69,10.20
-37.66190,0.00,18.100,0,0.6790,6.2020,78.70,1.8629,24,666.0,20.20,18.82,14.52,10.90
-7.36711,0.00,18.100,0,0.6790,6.1930,78.10,1.9356,24,666.0,20.20,96.73,21.52,11.00
-9.33889,0.00,18.100,0,0.6790,6.3800,95.60,1.9682,24,666.0,20.20,60.72,24.08,9.50
-8.49213,0.00,18.100,0,0.5840,6.3480,86.10,2.0527,24,666.0,20.20,83.45,17.64,14.50
-10.06230,0.00,18.100,0,0.5840,6.8330,94.30,2.0882,24,666.0,20.20,81.33,19.69,14.10
-6.44405,0.00,18.100,0,0.5840,6.4250,74.80,2.2004,24,666.0,20.20,97.95,12.03,16.10
-5.58107,0.00,18.100,0,0.7130,6.4360,87.90,2.3158,24,666.0,20.20,100.19,16.22,14.30
-13.91340,0.00,18.100,0,0.7130,6.2080,95.00,2.2222,24,666.0,20.20,100.63,15.17,11.70
-11.16040,0.00,18.100,0,0.7400,6.6290,94.60,2.1247,24,666.0,20.20,109.85,23.27,13.40
-14.42080,0.00,18.100,0,0.7400,6.4610,93.30,2.0026,24,666.0,20.20,27.49,18.05,9.60
-15.17720,0.00,18.100,0,0.7400,6.1520,100.00,1.9142,24,666.0,20.20,9.32,26.45,8.70
-13.67810,0.00,18.100,0,0.7400,5.9350,87.90,1.8206,24,666.0,20.20,68.95,34.02,8.40
-9.39063,0.00,18.100,0,0.7400,5.6270,93.90,1.8172,24,666.0,20.20,396.90,22.88,12.80
-22.05110,0.00,18.100,0,0.7400,5.8180,92.40,1.8662,24,666.0,20.20,391.45,22.11,10.50
-9.72418,0.00,18.100,0,0.7400,6.4060,97.20,2.0651,24,666.0,20.20,385.96,19.52,17.10
-5.66637,0.00,18.100,0,0.7400,6.2190,100.00,2.0048,24,666.0,20.20,395.69,16.59,18.40
-9.96654,0.00,18.100,0,0.7400,6.4850,100.00,1.9784,24,666.0,20.20,386.73,18.85,15.40
-12.80230,0.00,18.100,0,0.7400,5.8540,96.60,1.8956,24,666.0,20.20,240.52,23.79,10.80
-10.67180,0.00,18.100,0,0.7400,6.4590,94.80,1.9879,24,666.0,20.20,43.06,23.98,11.80
-6.28807,0.00,18.100,0,0.7400,6.3410,96.40,2.0720,24,666.0,20.20,318.01,17.79,14.90
-9.92485,0.00,18.100,0,0.7400,6.2510,96.60,2.1980,24,666.0,20.20,388.52,16.44,12.60
-9.32909,0.00,18.100,0,0.7130,6.1850,98.70,2.2616,24,666.0,20.20,396.90,18.13,14.10
-7.52601,0.00,18.100,0,0.7130,6.4170,98.30,2.1850,24,666.0,20.20,304.21,19.31,13.00
-6.71772,0.00,18.100,0,0.7130,6.7490,92.60,2.3236,24,666.0,20.20,0.32,17.44,13.40
-5.44114,0.00,18.100,0,0.7130,6.6550,98.20,2.3552,24,666.0,20.20,355.29,17.73,15.20
-5.09017,0.00,18.100,0,0.7130,6.2970,91.80,2.3682,24,666.0,20.20,385.09,17.27,16.10
-8.24809,0.00,18.100,0,0.7130,7.3930,99.30,2.4527,24,666.0,20.20,375.87,16.74,17.80
-9.51363,0.00,18.100,0,0.7130,6.7280,94.10,2.4961,24,666.0,20.20,6.68,18.71,14.90
-4.75237,0.00,18.100,0,0.7130,6.5250,86.50,2.4358,24,666.0,20.20,50.92,18.13,14.10
-4.66883,0.00,18.100,0,0.7130,5.9760,87.90,2.5806,24,666.0,20.20,10.48,19.01,12.70
-8.20058,0.00,18.100,0,0.7130,5.9360,80.30,2.7792,24,666.0,20.20,3.50,16.94,13.50
-7.75223,0.00,18.100,0,0.7130,6.3010,83.70,2.7831,24,666.0,20.20,272.21,16.23,14.90
-6.80117,0.00,18.100,0,0.7130,6.0810,84.40,2.7175,24,666.0,20.20,396.90,14.70,20.00
-4.81213,0.00,18.100,0,0.7130,6.7010,90.00,2.5975,24,666.0,20.20,255.23,16.42,16.40
-3.69311,0.00,18.100,0,0.7130,6.3760,88.40,2.5671,24,666.0,20.20,391.43,14.65,17.70
-6.65492,0.00,18.100,0,0.7130,6.3170,83.00,2.7344,24,666.0,20.20,396.90,13.99,19.50
-5.82115,0.00,18.100,0,0.7130,6.5130,89.90,2.8016,24,666.0,20.20,393.82,10.29,20.20
-7.83932,0.00,18.100,0,0.6550,6.2090,65.40,2.9634,24,666.0,20.20,396.90,13.22,21.40
-3.16360,0.00,18.100,0,0.6550,5.7590,48.20,3.0665,24,666.0,20.20,334.40,14.13,19.90
-3.77498,0.00,18.100,0,0.6550,5.9520,84.70,2.8715,24,666.0,20.20,22.01,17.15,19.00
-4.42228,0.00,18.100,0,0.5840,6.0030,94.50,2.5403,24,666.0,20.20,331.29,21.32,19.10
-15.57570,0.00,18.100,0,0.5800,5.9260,71.00,2.9084,24,666.0,20.20,368.74,18.13,19.10
-13.07510,0.00,18.100,0,0.5800,5.7130,56.70,2.8237,24,666.0,20.20,396.90,14.76,20.10
-4.34879,0.00,18.100,0,0.5800,6.1670,84.00,3.0334,24,666.0,20.20,396.90,16.29,19.90
-4.03841,0.00,18.100,0,0.5320,6.2290,90.70,3.0993,24,666.0,20.20,395.33,12.87,19.60
-3.56868,0.00,18.100,0,0.5800,6.4370,75.00,2.8965,24,666.0,20.20,393.37,14.36,23.20
-4.64689,0.00,18.100,0,0.6140,6.9800,67.60,2.5329,24,666.0,20.20,374.68,11.66,29.80
-8.05579,0.00,18.100,0,0.5840,5.4270,95.40,2.4298,24,666.0,20.20,352.58,18.14,13.80
-6.39312,0.00,18.100,0,0.5840,6.1620,97.40,2.2060,24,666.0,20.20,302.76,24.10,13.30
-4.87141,0.00,18.100,0,0.6140,6.4840,93.60,2.3053,24,666.0,20.20,396.21,18.68,16.70
-15.02340,0.00,18.100,0,0.6140,5.3040,97.30,2.1007,24,666.0,20.20,349.48,24.91,12.00
-10.23300,0.00,18.100,0,0.6140,6.1850,96.70,2.1705,24,666.0,20.20,379.70,18.03,14.60
-14.33370,0.00,18.100,0,0.6140,6.2290,88.00,1.9512,24,666.0,20.20,383.32,13.11,21.40
-5.82401,0.00,18.100,0,0.5320,6.2420,64.70,3.4242,24,666.0,20.20,396.90,10.74,23.00
-5.70818,0.00,18.100,0,0.5320,6.7500,74.90,3.3317,24,666.0,20.20,393.07,7.74,23.70
-5.73116,0.00,18.100,0,0.5320,7.0610,77.00,3.4106,24,666.0,20.20,395.28,7.01,25.00
-2.81838,0.00,18.100,0,0.5320,5.7620,40.30,4.0983,24,666.0,20.20,392.92,10.42,21.80
-2.37857,0.00,18.100,0,0.5830,5.8710,41.90,3.7240,24,666.0,20.20,370.73,13.34,20.60
-3.67367,0.00,18.100,0,0.5830,6.3120,51.90,3.9917,24,666.0,20.20,388.62,10.58,21.20
-5.69175,0.00,18.100,0,0.5830,6.1140,79.80,3.5459,24,666.0,20.20,392.68,14.98,19.10
-4.83567,0.00,18.100,0,0.5830,5.9050,53.20,3.1523,24,666.0,20.20,388.22,11.45,20.60
-0.15086,0.00,27.740,0,0.6090,5.4540,92.70,1.8209,4,711.0,20.10,395.09,18.06,15.20
-0.18337,0.00,27.740,0,0.6090,5.4140,98.30,1.7554,4,711.0,20.10,344.05,23.97,7.00
-0.20746,0.00,27.740,0,0.6090,5.0930,98.00,1.8226,4,711.0,20.10,318.43,29.68,8.10
-0.10574,0.00,27.740,0,0.6090,5.9830,98.80,1.8681,4,711.0,20.10,390.11,18.07,13.60
-0.11132,0.00,27.740,0,0.6090,5.9830,83.50,2.1099,4,711.0,20.10,396.90,13.35,20.10
-0.17331,0.00,9.690,0,0.5850,5.7070,54.00,2.3817,6,391.0,19.20,396.90,12.01,21.80
-0.27957,0.00,9.690,0,0.5850,5.9260,42.60,2.3817,6,391.0,19.20,396.90,13.59,24.50
-0.17899,0.00,9.690,0,0.5850,5.6700,28.80,2.7986,6,391.0,19.20,393.29,17.60,23.10
-0.28960,0.00,9.690,0,0.5850,5.3900,72.90,2.7986,6,391.0,19.20,396.90,21.14,19.70
-0.26838,0.00,9.690,0,0.5850,5.7940,70.60,2.8927,6,391.0,19.20,396.90,14.10,18.30
-0.23912,0.00,9.690,0,0.5850,6.0190,65.30,2.4091,6,391.0,19.20,396.90,12.92,21.20
-0.17783,0.00,9.690,0,0.5850,5.5690,73.50,2.3999,6,391.0,19.20,395.77,15.10,17.50
-0.22438,0.00,9.690,0,0.5850,6.0270,79.70,2.4982,6,391.0,19.20,396.90,14.33,16.80
-0.06263,0.00,11.930,0,0.5730,6.5930,69.10,2.4786,1,273.0,21.00,391.99,9.67,22.40
-0.04527,0.00,11.930,0,0.5730,6.1200,76.70,2.2875,1,273.0,21.00,396.90,9.08,20.60
-0.06076,0.00,11.930,0,0.5730,6.9760,91.00,2.1675,1,273.0,21.00,396.90,5.64,23.90
-0.10959,0.00,11.930,0,0.5730,6.7940,89.30,2.3889,1,273.0,21.00,393.45,6.48,22.00
-0.04741,0.00,11.930,0,0.5730,6.0300,80.80,2.5050,1,273.0,21.00,396.90,7.88,11.90

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/cleared_machines.csv
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/cleared_machines.csv b/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/cleared_machines.csv
deleted file mode 100644
index e22aac8..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/cleared_machines.csv
+++ /dev/null
@@ -1,209 +0,0 @@
-199;125;256;6000;256;16;128
-253;29;8000;32000;32;8;32
-253;29;8000;32000;32;8;32
-253;29;8000;32000;32;8;32
-132;29;8000;16000;32;8;16
-290;26;8000;32000;64;8;32
-381;23;16000;32000;64;16;32
-381;23;16000;32000;64;16;32
-749;23;16000;64000;64;16;32
-1238;23;32000;64000;128;32;64
-23;400;1000;3000;0;1;2
-24;400;512;3500;4;1;6
-70;60;2000;8000;65;1;8
-117;50;4000;16000;65;1;8
-15;350;64;64;0;1;4
-64;200;512;16000;0;4;32
-23;167;524;2000;8;4;15
-29;143;512;5000;0;7;32
-22;143;1000;2000;0;5;16
-124;110;5000;5000;142;8;64
-35;143;1500;6300;0;5;32
-39;143;3100;6200;0;5;20
-40;143;2300;6200;0;6;64
-45;110;3100;6200;0;6;64
-28;320;128;6000;0;1;12
-21;320;512;2000;4;1;3
-28;320;256;6000;0;1;6
-22;320;256;3000;4;1;3
-28;320;512;5000;4;1;5
-27;320;256;5000;4;1;6
-102;25;1310;2620;131;12;24
-102;25;1310;2620;131;12;24
-74;50;2620;10480;30;12;24
-74;50;2620;10480;30;12;24
-138;56;5240;20970;30;12;24
-136;64;5240;20970;30;12;24
-23;50;500;2000;8;1;4
-29;50;1000;4000;8;1;5
-44;50;2000;8000;8;1;5
-30;50;1000;4000;8;3;5
-41;50;1000;8000;8;3;5
-74;50;2000;16000;8;3;5
-74;50;2000;16000;8;3;6
-74;50;2000;16000;8;3;6
-54;133;1000;12000;9;3;12
-41;133;1000;8000;9;3;12
-18;810;512;512;8;1;1
-28;810;1000;5000;0;1;1
-36;320;512;8000;4;1;5
-38;200;512;8000;8;1;8
-34;700;384;8000;0;1;1
-19;700;256;2000;0;1;1
-72;140;1000;16000;16;1;3
-36;200;1000;8000;0;1;2
-30;110;1000;4000;16;1;2
-56;110;1000;12000;16;1;2
-42;220;1000;8000;16;1;2
-34;800;256;8000;0;1;4
-34;800;256;8000;0;1;4
-34;800;256;8000;0;1;4
-34;800;256;8000;0;1;4
-34;800;256;8000;0;1;4
-19;125;512;1000;0;8;20
-75;75;2000;8000;64;1;38
-113;75;2000;16000;64;1;38
-157;75;2000;16000;128;1;38
-18;90;256;1000;0;3;10
-20;105;256;2000;0;3;10
-28;105;1000;4000;0;3;24
-33;105;2000;4000;8;3;19
-47;75;2000;8000;8;3;24
-54;75;3000;8000;8;3;48
-20;175;256;2000;0;3;24
-23;300;768;3000;0;6;24
-25;300;768;3000;6;6;24
-52;300;768;12000;6;6;24
-27;300;768;4500;0;1;24
-50;300;384;12000;6;1;24
-18;300;192;768;6;6;24
-53;180;768;12000;6;1;31
-23;330;1000;3000;0;2;4
-30;300;1000;4000;8;3;64
-73;300;1000;16000;8;2;112
-20;330;1000;2000;0;1;2
-25;330;1000;4000;0;3;6
-28;140;2000;4000;0;3;6
-29;140;2000;4000;0;4;8
-32;140;2000;4000;8;1;20
-175;140;2000;32000;32;1;20
-57;140;2000;8000;32;1;54
-181;140;2000;32000;32;1;54
-181;140;2000;32000;32;1;54
-32;140;2000;4000;8;1;20
-82;57;4000;16000;1;6;12
-171;57;4000;24000;64;12;16
-361;26;16000;32000;64;16;24
-350;26;16000;32000;64;8;24
-220;26;8000;32000;0;8;24
-113;26;8000;16000;0;8;16
-15;480;96;512;0;1;1
-21;203;1000;2000;0;1;5
-35;115;512;6000;16;1;6
-18;1100;512;1500;0;1;1
-20;1100;768;2000;0;1;1
-20;600;768;2000;0;1;1
-28;400;2000;4000;0;1;1
-45;400;4000;8000;0;1;1
-18;900;1000;1000;0;1;2
-17;900;512;1000;0;1;2
-26;900;1000;4000;4;1;2
-28;900;1000;4000;8;1;2
-28;900;2000;4000;0;3;6
-31;225;2000;4000;8;3;6
-31;225;2000;4000;8;3;6
-42;180;2000;8000;8;1;6
-76;185;2000;16000;16;1;6
-76;180;2000;16000;16;1;6
-26;225;1000;4000;2;3;6
-59;25;2000;12000;8;1;4
-65;25;2000;12000;16;3;5
-101;17;4000;16000;8;6;12
-116;17;4000;16000;32;6;12
-18;1500;768;1000;0;0;0
-20;1500;768;2000;0;0;0
-20;800;768;2000;0;0;0
-30;50;2000;4000;0;3;6
-44;50;2000;8000;8;3;6
-44;50;2000;8000;8;1;6
-82;50;2000;16000;24;1;6
-82;50;2000;16000;24;1;6
-128;50;8000;16000;48;1;10
-37;100;1000;8000;0;2;6
-46;100;1000;8000;24;2;6
-46;100;1000;8000;24;3;6
-80;50;2000;16000;12;3;16
-88;50;2000;16000;24;6;16
-88;50;2000;16000;24;6;16
-33;150;512;4000;0;8;128
-46;115;2000;8000;16;1;3
-29;115;2000;4000;2;1;5
-53;92;2000;8000;32;1;6
-53;92;2000;8000;32;1;6
-41;92;2000;8000;4;1;6
-86;75;4000;16000;16;1;6
-95;60;4000;16000;32;1;6
-107;60;2000;16000;64;5;8
-117;60;4000;16000;64;5;8
-119;50;4000;16000;64;5;10
-120;72;4000;16000;64;8;16
-48;72;2000;8000;16;6;8
-126;40;8000;16000;32;8;16
-266;40;8000;32000;64;8;24
-270;35;8000;32000;64;8;24
-426;38;16000;32000;128;16;32
-151;48;4000;24000;32;8;24
-267;38;8000;32000;64;8;24
-603;30;16000;32000;256;16;24
-19;112;1000;1000;0;1;4
-21;84;1000;2000;0;1;6
-26;56;1000;4000;0;1;6
-35;56;2000;6000;0;1;8
-41;56;2000;8000;0;1;8
-47;56;4000;8000;0;1;8
-62;56;4000;12000;0;1;8
-78;56;4000;16000;0;1;8
-80;38;4000;8000;32;16;32
-80;38;4000;8000;32;16;32
-142;38;8000;16000;64;4;8
-281;38;8000;24000;160;4;8
-190;38;4000;16000;128;16;32
-21;200;1000;2000;0;1;2
-25;200;1000;4000;0;1;4
-67;200;2000;8000;64;1;5
-24;250;512;4000;0;1;7
-24;250;512;4000;0;4;7
-64;250;1000;16000;1;1;8
-25;160;512;4000;2;1;5
-20;160;512;2000;2;3;8
-29;160;1000;4000;8;1;14
-43;160;1000;8000;16;1;14
-53;160;2000;8000;32;1;13
-19;240;512;1000;8;1;3
-22;240;512;2000;8;1;5
-31;105;2000;4000;8;3;8
-41;105;2000;6000;16;6;16
-47;105;2000;8000;16;4;14
-99;52;4000;16000;32;4;12
-67;70;4000;12000;8;6;8
-81;59;4000;12000;32;6;12
-149;59;8000;16000;64;12;24
-183;26;8000;24000;32;8;16
-275;26;8000;32000;64;12;16
-382;26;8000;32000;128;24;32
-56;116;2000;8000;32;5;28
-182;50;2000;32000;24;6;26
-227;50;2000;32000;48;26;52
-341;50;2000;32000;112;52;104
-360;50;4000;32000;112;52;104
-919;30;8000;64000;96;12;176
-978;30;8000;64000;128;12;176
-24;180;262;4000;0;1;3
-24;180;512;4000;0;1;3
-24;180;262;4000;0;1;3
-24;180;512;4000;0;1;3
-37;124;1000;8000;0;1;8
-50;98;1000;8000;32;2;8
-41;125;2000;8000;0;2;14
-47;480;512;8000;32;0;0
-25;480;1000;4000;0;0;0

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/glass_identification.csv
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/glass_identification.csv b/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/glass_identification.csv
deleted file mode 100644
index ae1d6d1..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/glass_identification.csv
+++ /dev/null
@@ -1,116 +0,0 @@
-1; 1.52101; 4.49; 1.10; 0.00; 0.00
-1; 1.51761; 3.60; 1.36; 0.00; 0.00
-1; 1.51618; 3.55; 1.54; 0.00; 0.00
-1; 1.51766; 3.69; 1.29; 0.00; 0.00
-1; 1.51742; 3.62; 1.24; 0.00; 0.00
-1; 1.51596; 3.61; 1.62; 0.00; 0.26
-1; 1.51743; 3.60; 1.14; 0.00; 0.00
-1; 1.51756; 3.61; 1.05; 0.00; 0.00
-1; 1.51918; 3.58; 1.37; 0.00; 0.00
-1; 1.51755; 3.60; 1.36; 0.00; 0.11
-1; 1.51571; 3.46; 1.56; 0.00; 0.24
-1; 1.51763; 3.66; 1.27; 0.00; 0.00
-1; 1.51589; 3.43; 1.40; 0.00; 0.24
-1; 1.51748; 3.56; 1.27; 0.00; 0.17
-1; 1.51763; 3.59; 1.31; 0.00; 0.00
-1; 1.51761; 3.54; 1.23; 0.00; 0.00
-1; 1.51784; 3.67; 1.16; 0.00; 0.00
-1; 1.52196; 3.85; 0.89; 0.00; 0.00
-1; 1.51911; 3.73; 1.18; 0.00; 0.00
-1; 1.51735; 3.54; 1.69; 0.00; 0.07
-1; 1.51750; 3.55; 1.49; 0.00; 0.19
-1; 1.51966; 3.75; 0.29; 0.00; 0.00
-1; 1.51736; 3.62; 1.29; 0.00; 0.00
-1; 1.51751; 3.57; 1.35; 0.00; 0.00
-1; 1.51720; 3.50; 1.15; 0.00; 0.00
-1; 1.51764; 3.54; 1.21; 0.00; 0.00
-1; 1.51793; 3.48; 1.41; 0.00; 0.00
-1; 1.51721; 3.48; 1.33; 0.00; 0.00
-1; 1.51768; 3.52; 1.43; 0.00; 0.00
-1; 1.51784; 3.49; 1.28; 0.00; 0.00
-1; 1.51768; 3.56; 1.30; 0.00; 0.14
-1; 1.51747; 3.50; 1.14; 0.00; 0.00
-1; 1.51775; 3.48; 1.23; 0.09; 0.22
-1; 1.51753; 3.47; 1.38; 0.00; 0.06
-1; 1.51783; 3.54; 1.34; 0.00; 0.00
-1; 1.51567; 3.45; 1.21; 0.00; 0.00
-1; 1.51909; 3.53; 1.32; 0.11; 0.00
-1; 1.51797; 3.48; 1.35; 0.00; 0.00
-1; 1.52213; 3.82; 0.47; 0.00; 0.00
-1; 1.52213; 3.82; 0.47; 0.00; 0.00
-1; 1.51793; 3.50; 1.12; 0.00; 0.00
-1; 1.51755; 3.42; 1.20; 0.00; 0.00
-1; 1.51779; 3.39; 1.33; 0.00; 0.00
-1; 1.52210; 3.84; 0.72; 0.00; 0.00
-1; 1.51786; 3.43; 1.19; 0.00; 0.30
-1; 1.51900; 3.48; 1.35; 0.00; 0.00
-1; 1.51869; 3.37; 1.18; 0.00; 0.16
-1; 1.52667; 3.70; 0.71; 0.00; 0.10
-1; 1.52223; 3.77; 0.79; 0.00; 0.00
-1; 1.51898; 3.35; 1.23; 0.00; 0.00
-1; 1.52320; 3.72; 0.51; 0.00; 0.16
-1; 1.51926; 3.33; 1.28; 0.00; 0.11
-1; 1.51808; 2.87; 1.19; 0.00; 0.00
-1; 1.51837; 2.84; 1.28; 0.00; 0.00
-1; 1.51778; 2.81; 1.29; 0.00; 0.09
-1; 1.51769; 2.71; 1.29; 0.00; 0.24
-1; 1.51215; 3.47; 1.12; 0.00; 0.31
-1; 1.51824; 3.48; 1.29; 0.00; 0.00
-1; 1.51754; 3.74; 1.17; 0.00; 0.00
-1; 1.51754; 3.66; 1.19; 0.00; 0.11
-1; 1.51905; 3.62; 1.11; 0.00; 0.00
-1; 1.51977; 3.58; 1.32; 0.69; 0.00
-1; 1.52172; 3.86; 0.88; 0.00; 0.11
-1; 1.52227; 3.81; 0.78; 0.00; 0.00
-1; 1.52172; 3.74; 0.90; 0.00; 0.07
-1; 1.52099; 3.59; 1.12; 0.00; 0.00
-1; 1.52152; 3.65; 0.87; 0.00; 0.17
-1; 1.52152; 3.65; 0.87; 0.00; 0.17
-1; 1.52152; 3.58; 0.90; 0.00; 0.16
-1; 1.52300; 3.58; 0.82; 0.00; 0.03
-3; 1.51769; 3.66; 1.11; 0.00; 0.00
-3; 1.51610; 3.53; 1.34; 0.00; 0.00
-3; 1.51670; 3.57; 1.38; 0.00; 0.10
-3; 1.51643; 3.52; 1.35; 0.00; 0.00
-3; 1.51665; 3.45; 1.76; 0.00; 0.17
-3; 1.52127; 3.90; 0.83; 0.00; 0.00
-3; 1.51779; 3.65; 0.65; 0.00; 0.00
-3; 1.51610; 3.40; 1.22; 0.00; 0.00
-3; 1.51694; 3.58; 1.31; 0.00; 0.00
-3; 1.51646; 3.40; 1.26; 0.00; 0.00
-3; 1.51655; 3.39; 1.28; 0.00; 0.00
-3; 1.52121; 3.76; 0.58; 0.00; 0.00
-3; 1.51776; 3.41; 1.52; 0.00; 0.00
-3; 1.51796; 3.36; 1.63; 0.00; 0.09
-3; 1.51832; 3.34; 1.54; 0.00; 0.00
-3; 1.51934; 3.54; 0.75; 0.15; 0.24
-3; 1.52211; 3.78; 0.91; 0.00; 0.37
-7; 1.51131; 3.20; 1.81; 1.19; 0.00
-7; 1.51838; 3.26; 2.22; 1.63; 0.00
-7; 1.52315; 3.34; 1.23; 0.00; 0.00
-7; 1.52247; 2.20; 2.06; 0.00; 0.00
-7; 1.52365; 1.83; 1.31; 1.68; 0.00
-7; 1.51613; 1.78; 1.79; 0.76; 0.00
-7; 1.51602; 0.00; 2.38; 0.64; 0.09
-7; 1.51623; 0.00; 2.79; 0.40; 0.09
-7; 1.51719; 0.00; 2.00; 1.59; 0.08
-7; 1.51683; 0.00; 1.98; 1.57; 0.07
-7; 1.51545; 0.00; 2.68; 0.61; 0.05
-7; 1.51556; 0.00; 2.54; 0.81; 0.01
-7; 1.51727; 0.00; 2.34; 0.66; 0.00
-7; 1.51531; 0.00; 2.66; 0.64; 0.00
-7; 1.51609; 0.00; 2.51; 0.53; 0.00
-7; 1.51508; 0.00; 2.25; 0.63; 0.00
-7; 1.51653; 0.00; 1.19; 0.00; 0.00
-7; 1.51514; 0.00; 2.42; 0.56; 0.00
-7; 1.51658; 0.00; 1.99; 1.71; 0.00
-7; 1.51617; 0.00; 2.27; 0.67; 0.00
-7; 1.51732; 0.00; 1.80; 1.55; 0.00
-7; 1.51645; 0.00; 1.87; 1.38; 0.00
-7; 1.51831; 0.00; 1.82; 2.88; 0.00
-7; 1.51640; 0.00; 2.74; 0.54; 0.00
-7; 1.51623; 0.00; 2.88; 1.06; 0.00
-7; 1.51685; 0.00; 1.99; 1.59; 0.00
-7; 1.52065; 0.00; 2.02; 1.64; 0.00
-7; 1.51651; 0.00; 1.94; 1.57; 0.00
-7; 1.51711; 0.00; 2.08; 1.67; 0.00

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/iris.txt
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/iris.txt b/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/iris.txt
deleted file mode 100644
index 18f5f7c..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/iris.txt
+++ /dev/null
@@ -1,150 +0,0 @@
-1.0	5.1	3.5	1.4	0.2
-1.0	4.9	3.0	1.4	0.2
-1.0	4.7	3.2	1.3	0.2
-1.0	4.6	3.1	1.5	0.2
-1.0	5.0	3.6	1.4	0.2
-1.0	5.4	3.9	1.7	0.4
-1.0	4.6	3.4	1.4	0.3
-1.0	5.0	3.4	1.5	0.2
-1.0	4.4	2.9	1.4	0.2
-1.0	4.9	3.1	1.5	0.1
-1.0	5.4	3.7	1.5	0.2
-1.0	4.8	3.4	1.6	0.2
-1.0	4.8	3.0	1.4	0.1
-1.0	4.3	3.0	1.1	0.1
-1.0	5.8	4.0	1.2	0.2
-1.0	5.7	4.4	1.5	0.4
-1.0	5.4	3.9	1.3	0.4
-1.0	5.1	3.5	1.4	0.3
-1.0	5.7	3.8	1.7	0.3
-1.0	5.1	3.8	1.5	0.3
-1.0	5.4	3.4	1.7	0.2
-1.0	5.1	3.7	1.5	0.4
-1.0	4.6	3.6	1.0	0.2
-1.0	5.1	3.3	1.7	0.5
-1.0	4.8	3.4	1.9	0.2
-1.0	5.0	3.0	1.6	0.2
-1.0	5.0	3.4	1.6	0.4
-1.0	5.2	3.5	1.5	0.2
-1.0	5.2	3.4	1.4	0.2
-1.0	4.7	3.2	1.6	0.2
-1.0	4.8	3.1	1.6	0.2
-1.0	5.4	3.4	1.5	0.4
-1.0	5.2	4.1	1.5	0.1
-1.0	5.5	4.2	1.4	0.2
-1.0	4.9	3.1	1.5	0.1
-1.0	5.0	3.2	1.2	0.2
-1.0	5.5	3.5	1.3	0.2
-1.0	4.9	3.1	1.5	0.1
-1.0	4.4	3.0	1.3	0.2
-1.0	5.1	3.4	1.5	0.2
-1.0	5.0	3.5	1.3	0.3
-1.0	4.5	2.3	1.3	0.3
-1.0	4.4	3.2	1.3	0.2
-1.0	5.0	3.5	1.6	0.6
-1.0	5.1	3.8	1.9	0.4
-1.0	4.8	3.0	1.4	0.3
-1.0	5.1	3.8	1.6	0.2
-1.0	4.6	3.2	1.4	0.2
-1.0	5.3	3.7	1.5	0.2
-1.0	5.0	3.3	1.4	0.2
-2.0	7.0	3.2	4.7	1.4
-2.0	6.4	3.2	4.5	1.5
-2.0	6.9	3.1	4.9	1.5
-2.0	5.5	2.3	4.0	1.3
-2.0	6.5	2.8	4.6	1.5
-2.0	5.7	2.8	4.5	1.3
-2.0	6.3	3.3	4.7	1.6
-2.0	4.9	2.4	3.3	1.0
-2.0	6.6	2.9	4.6	1.3
-2.0	5.2	2.7	3.9	1.4
-2.0	5.0	2.0	3.5	1.0
-2.0	5.9	3.0	4.2	1.5
-2.0	6.0	2.2	4.0	1.0
-2.0	6.1	2.9	4.7	1.4
-2.0	5.6	2.9	3.6	1.3
-2.0	6.7	3.1	4.4	1.4
-2.0	5.6	3.0	4.5	1.5
-2.0	5.8	2.7	4.1	1.0
-2.0	6.2	2.2	4.5	1.5
-2.0	5.6	2.5	3.9	1.1
-2.0	5.9	3.2	4.8	1.8
-2.0	6.1	2.8	4.0	1.3
-2.0	6.3	2.5	4.9	1.5
-2.0	6.1	2.8	4.7	1.2
-2.0	6.4	2.9	4.3	1.3
-2.0	6.6	3.0	4.4	1.4
-2.0	6.8	2.8	4.8	1.4
-2.0	6.7	3.0	5.0	1.7
-2.0	6.0	2.9	4.5	1.5
-2.0	5.7	2.6	3.5	1.0
-2.0	5.5	2.4	3.8	1.1
-2.0	5.5	2.4	3.7	1.0
-2.0	5.8	2.7	3.9	1.2
-2.0	6.0	2.7	5.1	1.6
-2.0	5.4	3.0	4.5	1.5
-2.0	6.0	3.4	4.5	1.6
-2.0	6.7	3.1	4.7	1.5
-2.0	6.3	2.3	4.4	1.3
-2.0	5.6	3.0	4.1	1.3
-2.0	5.5	2.5	4.0	1.3
-2.0	5.5	2.6	4.4	1.2
-2.0	6.1	3.0	4.6	1.4
-2.0	5.8	2.6	4.0	1.2
-2.0	5.0	2.3	3.3	1.0
-2.0	5.6	2.7	4.2	1.3
-2.0	5.7	3.0	4.2	1.2
-2.0	5.7	2.9	4.2	1.3
-2.0	6.2	2.9	4.3	1.3
-2.0	5.1	2.5	3.0	1.1
-2.0	5.7	2.8	4.1	1.3
-3.0	6.3	3.3	6.0	2.5
-3.0	5.8	2.7	5.1	1.9
-3.0	7.1	3.0	5.9	2.1
-3.0	6.3	2.9	5.6	1.8
-3.0	6.5	3.0	5.8	2.2
-3.0	7.6	3.0	6.6	2.1
-3.0	4.9	2.5	4.5	1.7
-3.0	7.3	2.9	6.3	1.8
-3.0	6.7	2.5	5.8	1.8
-3.0	7.2	3.6	6.1	2.5
-3.0	6.5	3.2	5.1	2.0
-3.0	6.4	2.7	5.3	1.9
-3.0	6.8	3.0	5.5	2.1
-3.0	5.7	2.5	5.0	2.0
-3.0	5.8	2.8	5.1	2.4
-3.0	6.4	3.2	5.3	2.3
-3.0	6.5	3.0	5.5	1.8
-3.0	7.7	3.8	6.7	2.2
-3.0	7.7	2.6	6.9	2.3
-3.0	6.0	2.2	5.0	1.5
-3.0	6.9	3.2	5.7	2.3
-3.0	5.6	2.8	4.9	2.0
-3.0	7.7	2.8	6.7	2.0
-3.0	6.3	2.7	4.9	1.8
-3.0	6.7	3.3	5.7	2.1
-3.0	7.2	3.2	6.0	1.8
-3.0	6.2	2.8	4.8	1.8
-3.0	6.1	3.0	4.9	1.8
-3.0	6.4	2.8	5.6	2.1
-3.0	7.2	3.0	5.8	1.6
-3.0	7.4	2.8	6.1	1.9
-3.0	7.9	3.8	6.4	2.0
-3.0	6.4	2.8	5.6	2.2
-3.0	6.3	2.8	5.1	1.5
-3.0	6.1	2.6	5.6	1.4
-3.0	7.7	3.0	6.1	2.3
-3.0	6.3	3.4	5.6	2.4
-3.0	6.4	3.1	5.5	1.8
-3.0	6.0	3.0	4.8	1.8
-3.0	6.9	3.1	5.4	2.1
-3.0	6.7	3.1	5.6	2.4
-3.0	6.9	3.1	5.1	2.3
-3.0	5.8	2.7	5.1	1.9
-3.0	6.8	3.2	5.9	2.3
-3.0	6.7	3.3	5.7	2.5
-3.0	6.7	3.0	5.2	2.3
-3.0	6.3	2.5	5.0	1.9
-3.0	6.5	3.0	5.2	2.0
-3.0	6.2	3.4	5.4	2.3
-3.0	5.9	3.0	5.1	1.8

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/mortalitydata.csv
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/mortalitydata.csv b/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/mortalitydata.csv
deleted file mode 100644
index e4f3e41..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/mortalitydata.csv
+++ /dev/null
@@ -1,53 +0,0 @@
-8; 78; 284; 9.100000381; 109
-9.300000191; 68; 433; 8.699999809; 144
-7.5; 70; 739; 7.199999809; 113
-8.899999619; 96; 1792; 8.899999619; 97
-10.19999981; 74; 477; 8.300000191; 206
-8.300000191; 111; 362; 10.89999962; 124
-8.800000191; 77; 671; 10; 152
-8.800000191; 168; 636; 9.100000381; 162
-10.69999981; 82; 329; 8.699999809; 150
-11.69999981; 89; 634; 7.599999905; 134
-8.5; 149; 631; 10.80000019; 292
-8.300000191; 60; 257; 9.5; 108
-8.199999809; 96; 284; 8.800000191; 111
-7.900000095; 83; 603; 9.5; 182
-10.30000019; 130; 686; 8.699999809; 129
-7.400000095; 145; 345; 11.19999981; 158
-9.600000381; 112; 1357; 9.699999809; 186
-9.300000191; 131; 544; 9.600000381; 177
-10.60000038; 80; 205; 9.100000381; 127
-9.699999809; 130; 1264; 9.199999809; 179
-11.60000038; 140; 688; 8.300000191; 80
-8.100000381; 154; 354; 8.399999619; 103
-9.800000191; 118; 1632; 9.399999619; 101
-7.400000095; 94; 348; 9.800000191; 117
-9.399999619; 119; 370; 10.39999962; 88
-11.19999981; 153; 648; 9.899999619; 78
-9.100000381; 116; 366; 9.199999809; 102
-10.5; 97; 540; 10.30000019; 95
-11.89999962; 176; 680; 8.899999619; 80
-8.399999619; 75; 345; 9.600000381; 92
-5; 134; 525; 10.30000019; 126
-9.800000191; 161; 870; 10.39999962; 108
-9.800000191; 111; 669; 9.699999809; 77
-10.80000019; 114; 452; 9.600000381; 60
-10.10000038; 142; 430; 10.69999981; 71
-10.89999962; 238; 822; 10.30000019; 86
-9.199999809; 78; 190; 10.69999981; 93
-8.300000191; 196; 867; 9.600000381; 106
-7.300000191; 125; 969; 10.5; 162
-9.399999619; 82; 499; 7.699999809; 95
-9.399999619; 125; 925; 10.19999981; 91
-9.800000191; 129; 353; 9.899999619; 52
-3.599999905; 84; 288; 8.399999619; 110
-8.399999619; 183; 718; 10.39999962; 69
-10.80000019; 119; 540; 9.199999809; 57
-10.10000038; 180; 668; 13; 106
-9; 82; 347; 8.800000191; 40
-10; 71; 345; 9.199999809; 50
-11.30000019; 118; 463; 7.800000191; 35
-11.30000019; 121; 728; 8.199999809; 86
-12.80000019; 68; 383; 7.400000095; 57
-10; 112; 316; 10.39999962; 57
-6.699999809; 109; 388; 8.899999619; 94

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/t10k-images-idx3-ubyte
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/t10k-images-idx3-ubyte b/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/t10k-images-idx3-ubyte
deleted file mode 100644
index 1170b2c..0000000
Binary files a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/t10k-images-idx3-ubyte and /dev/null differ

http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/t10k-labels-idx1-ubyte
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/t10k-labels-idx1-ubyte b/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/t10k-labels-idx1-ubyte
deleted file mode 100644
index d1c3a97..0000000
Binary files a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/t10k-labels-idx1-ubyte and /dev/null differ


[23/50] [abbrv] ignite git commit: IGNITE-10358: Added collections data type specification for python thin client

Posted by ag...@apache.org.
IGNITE-10358: Added collections data type specification for python thin client

This closes #5470


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

Branch: refs/heads/ignite-9720
Commit: 04fae6d2cbbd83b15a43cf17da9011c0eccec41b
Parents: acc1024
Author: Dmitry Melnichuk <dm...@nobitlost.com>
Authored: Mon Nov 26 17:14:59 2018 +0300
Committer: Igor Sapego <is...@apache.org>
Committed: Mon Nov 26 17:34:19 2018 +0300

----------------------------------------------------------------------
 .../docs/source/pyignite.datatypes.base.rst     |  7 ++
 .../python/docs/source/pyignite.datatypes.rst   |  1 +
 .../platforms/python/pyignite/datatypes/base.py | 24 +++++++
 .../python/pyignite/datatypes/complex.py        | 23 +++---
 .../python/pyignite/datatypes/internal.py       | 19 +++--
 .../python/pyignite/datatypes/null_object.py    |  3 +-
 .../python/pyignite/datatypes/primitive.py      |  3 +-
 .../pyignite/datatypes/primitive_arrays.py      |  3 +-
 .../pyignite/datatypes/primitive_objects.py     |  3 +-
 .../python/pyignite/datatypes/standard.py       |  9 +--
 modules/platforms/python/pyignite/utils.py      |  6 +-
 modules/platforms/python/setup.py               |  2 +-
 .../platforms/python/tests/test_key_value.py    | 75 +++++++++++++++++++-
 13 files changed, 149 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/04fae6d2/modules/platforms/python/docs/source/pyignite.datatypes.base.rst
----------------------------------------------------------------------
diff --git a/modules/platforms/python/docs/source/pyignite.datatypes.base.rst b/modules/platforms/python/docs/source/pyignite.datatypes.base.rst
new file mode 100644
index 0000000..849a028
--- /dev/null
+++ b/modules/platforms/python/docs/source/pyignite.datatypes.base.rst
@@ -0,0 +1,7 @@
+pyignite.datatypes.base module
+==============================
+
+.. automodule:: pyignite.datatypes.base
+    :members:
+    :undoc-members:
+    :show-inheritance:

http://git-wip-us.apache.org/repos/asf/ignite/blob/04fae6d2/modules/platforms/python/docs/source/pyignite.datatypes.rst
----------------------------------------------------------------------
diff --git a/modules/platforms/python/docs/source/pyignite.datatypes.rst b/modules/platforms/python/docs/source/pyignite.datatypes.rst
index 77e7183..d72f844 100644
--- a/modules/platforms/python/docs/source/pyignite.datatypes.rst
+++ b/modules/platforms/python/docs/source/pyignite.datatypes.rst
@@ -11,6 +11,7 @@ Submodules
 
 .. toctree::
 
+   pyignite.datatypes.base
    pyignite.datatypes.binary
    pyignite.datatypes.cache_config
    pyignite.datatypes.cache_properties

http://git-wip-us.apache.org/repos/asf/ignite/blob/04fae6d2/modules/platforms/python/pyignite/datatypes/base.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/base.py b/modules/platforms/python/pyignite/datatypes/base.py
new file mode 100644
index 0000000..a0522c0
--- /dev/null
+++ b/modules/platforms/python/pyignite/datatypes/base.py
@@ -0,0 +1,24 @@
+# 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.
+
+from abc import ABC
+
+
+class IgniteDataType(ABC):
+    """
+    This is a base class for all Ignite data types, a.k.a. parser/constructor
+    classes, both object and payload varieties.
+    """
+    pass

http://git-wip-us.apache.org/repos/asf/ignite/blob/04fae6d2/modules/platforms/python/pyignite/datatypes/complex.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/complex.py b/modules/platforms/python/pyignite/datatypes/complex.py
index 9a5664c..87e5130 100644
--- a/modules/platforms/python/pyignite/datatypes/complex.py
+++ b/modules/platforms/python/pyignite/datatypes/complex.py
@@ -20,7 +20,8 @@ import inspect
 from pyignite.constants import *
 from pyignite.exceptions import ParseError
 from pyignite.utils import entity_id, hashcode, is_hinted
-from .internal import AnyDataObject
+from .base import IgniteDataType
+from .internal import AnyDataObject, infer_from_python
 from .type_codes import *
 
 
@@ -30,7 +31,7 @@ __all__ = [
 ]
 
 
-class ObjectArrayObject:
+class ObjectArrayObject(IgniteDataType):
     """
     Array of objects of any type. Its Python representation is
     tuple(type_id, iterable of any type).
@@ -106,11 +107,11 @@ class ObjectArrayObject:
         buffer = bytes(header)
 
         for x in value:
-            buffer += AnyDataObject.from_python(x)
+            buffer += infer_from_python(x)
         return buffer
 
 
-class WrappedDataObject:
+class WrappedDataObject(IgniteDataType):
     """
     One or more binary objects can be wrapped in an array. This allows reading,
     storing, passing and writing objects efficiently without understanding
@@ -195,7 +196,7 @@ class CollectionObject(ObjectArrayObject):
         )
 
 
-class Map:
+class Map(IgniteDataType):
     """
     Dictionary type, payload-only.
 
@@ -273,14 +274,8 @@ class Map:
         buffer = bytes(header)
 
         for k, v in value.items():
-            if is_hinted(k):
-                buffer += k[1].from_python(k[0])
-            else:
-                buffer += AnyDataObject.from_python(k)
-            if is_hinted(v):
-                buffer += v[1].from_python(v[0])
-            else:
-                buffer += AnyDataObject.from_python(v)
+            buffer += infer_from_python(k)
+            buffer += infer_from_python(v)
         return buffer
 
 
@@ -323,7 +318,7 @@ class MapObject(Map):
         return super().from_python(value, type_id)
 
 
-class BinaryObject:
+class BinaryObject(IgniteDataType):
     type_code = TC_COMPLEX_OBJECT
 
     USER_TYPE = 0x0001

http://git-wip-us.apache.org/repos/asf/ignite/blob/04fae6d2/modules/platforms/python/pyignite/datatypes/internal.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/internal.py b/modules/platforms/python/pyignite/datatypes/internal.py
index a363a5f..844e0ef 100644
--- a/modules/platforms/python/pyignite/datatypes/internal.py
+++ b/modules/platforms/python/pyignite/datatypes/internal.py
@@ -389,6 +389,20 @@ class AnyDataObject:
         return cls.map_python_type(value).from_python(value)
 
 
+def infer_from_python(value: Any):
+    """
+    Convert pythonic value to ctypes buffer, type hint-aware.
+
+    :param value: pythonic value or (value, type_hint) tuple,
+    :return: bytes.
+    """
+    if is_hinted(value):
+        value, data_type = value
+    else:
+        data_type = AnyDataObject
+    return data_type.from_python(value)
+
+
 @attr.s
 class AnyDataArray(AnyDataObject):
     """
@@ -454,8 +468,5 @@ class AnyDataArray(AnyDataObject):
         buffer = bytes(header)
 
         for x in value:
-            if is_hinted(x):
-                buffer += x[1].from_python(x[0])
-            else:
-                buffer += super().from_python(x)
+            buffer += infer_from_python(x)
         return buffer

http://git-wip-us.apache.org/repos/asf/ignite/blob/04fae6d2/modules/platforms/python/pyignite/datatypes/null_object.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/null_object.py b/modules/platforms/python/pyignite/datatypes/null_object.py
index 9fa1e8f..a648e30 100644
--- a/modules/platforms/python/pyignite/datatypes/null_object.py
+++ b/modules/platforms/python/pyignite/datatypes/null_object.py
@@ -21,13 +21,14 @@ There can't be null type, because null payload takes exactly 0 bytes.
 
 import ctypes
 
+from .base import IgniteDataType
 from .type_codes import TC_NULL
 
 
 __all__ = ['Null']
 
 
-class Null:
+class Null(IgniteDataType):
     default = None
     pythonic = type(None)
     _object_c_type = None

http://git-wip-us.apache.org/repos/asf/ignite/blob/04fae6d2/modules/platforms/python/pyignite/datatypes/primitive.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/primitive.py b/modules/platforms/python/pyignite/datatypes/primitive.py
index 94c8fe3..d1e9f4e 100644
--- a/modules/platforms/python/pyignite/datatypes/primitive.py
+++ b/modules/platforms/python/pyignite/datatypes/primitive.py
@@ -16,6 +16,7 @@
 import ctypes
 
 from pyignite.constants import *
+from .base import IgniteDataType
 
 
 __all__ = [
@@ -24,7 +25,7 @@ __all__ = [
 ]
 
 
-class Primitive:
+class Primitive(IgniteDataType):
     """
     Ignite primitive type. Base type for the following types:
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/04fae6d2/modules/platforms/python/pyignite/datatypes/primitive_arrays.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/primitive_arrays.py b/modules/platforms/python/pyignite/datatypes/primitive_arrays.py
index 83a2b4c..6a93191 100644
--- a/modules/platforms/python/pyignite/datatypes/primitive_arrays.py
+++ b/modules/platforms/python/pyignite/datatypes/primitive_arrays.py
@@ -16,6 +16,7 @@
 import ctypes
 
 from pyignite.constants import *
+from .base import IgniteDataType
 from .primitive import *
 from .type_codes import *
 
@@ -28,7 +29,7 @@ __all__ = [
 ]
 
 
-class PrimitiveArray:
+class PrimitiveArray(IgniteDataType):
     """
     Base class for array of primitives. Payload-only.
     """

http://git-wip-us.apache.org/repos/asf/ignite/blob/04fae6d2/modules/platforms/python/pyignite/datatypes/primitive_objects.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/primitive_objects.py b/modules/platforms/python/pyignite/datatypes/primitive_objects.py
index 4e37ce1..105acee 100644
--- a/modules/platforms/python/pyignite/datatypes/primitive_objects.py
+++ b/modules/platforms/python/pyignite/datatypes/primitive_objects.py
@@ -16,6 +16,7 @@
 import ctypes
 
 from pyignite.constants import *
+from .base import IgniteDataType
 from .type_codes import *
 
 
@@ -25,7 +26,7 @@ __all__ = [
 ]
 
 
-class DataObject:
+class DataObject(IgniteDataType):
     """
     Base class for primitive data objects.
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/04fae6d2/modules/platforms/python/pyignite/datatypes/standard.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/standard.py b/modules/platforms/python/pyignite/datatypes/standard.py
index 5f3af74..cc5b955 100644
--- a/modules/platforms/python/pyignite/datatypes/standard.py
+++ b/modules/platforms/python/pyignite/datatypes/standard.py
@@ -20,6 +20,7 @@ from math import ceil
 import uuid
 
 from pyignite.constants import *
+from .base import IgniteDataType
 from .type_codes import *
 from .null_object import Null
 
@@ -39,7 +40,7 @@ __all__ = [
 ]
 
 
-class StandardObject:
+class StandardObject(IgniteDataType):
     type_code = None
 
     @classmethod
@@ -58,7 +59,7 @@ class StandardObject:
         return c_type, buffer
 
 
-class String:
+class String(IgniteDataType):
     """
     Pascal-style string: `c_int` counter, followed by count*bytes.
     UTF-8-encoded, so that one character may take 1 to 4 bytes.
@@ -125,7 +126,7 @@ class String:
         return bytes(data_object)
 
 
-class DecimalObject:
+class DecimalObject(IgniteDataType):
     type_code = TC_DECIMAL
     pythonic = decimal.Decimal
     default = decimal.Decimal('0.00')
@@ -511,7 +512,7 @@ class BinaryEnumObject(EnumObject):
     type_code = TC_BINARY_ENUM
 
 
-class StandardArray:
+class StandardArray(IgniteDataType):
     """
     Base class for array of primitives. Payload-only.
     """

http://git-wip-us.apache.org/repos/asf/ignite/blob/04fae6d2/modules/platforms/python/pyignite/utils.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/utils.py b/modules/platforms/python/pyignite/utils.py
index a08bc9b..1d4298e 100644
--- a/modules/platforms/python/pyignite/utils.py
+++ b/modules/platforms/python/pyignite/utils.py
@@ -16,6 +16,7 @@
 from functools import wraps
 from typing import Any, Type, Union
 
+from pyignite.datatypes.base import IgniteDataType
 from .constants import *
 
 
@@ -47,11 +48,14 @@ def is_hinted(value):
     return (
         isinstance(value, tuple)
         and len(value) == 2
-        and isinstance(value[1], object)
+        and issubclass(value[1], IgniteDataType)
     )
 
 
 def is_wrapped(value: Any) -> bool:
+    """
+    Check if a value is of WrappedDataObject type.
+    """
     return (
         type(value) is tuple
         and len(value) == 2

http://git-wip-us.apache.org/repos/asf/ignite/blob/04fae6d2/modules/platforms/python/setup.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/setup.py b/modules/platforms/python/setup.py
index 7419c97..583eaa3 100644
--- a/modules/platforms/python/setup.py
+++ b/modules/platforms/python/setup.py
@@ -70,7 +70,7 @@ with open('README.md', 'r', encoding='utf-8') as readme_file:
 
 setuptools.setup(
     name='pyignite',
-    version='0.3.1',
+    version='0.3.4',
     python_requires='>={}.{}'.format(*PYTHON_REQUIRED),
     author='Dmitry Melnichuk',
     author_email='dmitry.melnichuk@nobitlost.com',

http://git-wip-us.apache.org/repos/asf/ignite/blob/04fae6d2/modules/platforms/python/tests/test_key_value.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/tests/test_key_value.py b/modules/platforms/python/tests/test_key_value.py
index c569c77..6b4fb0e 100644
--- a/modules/platforms/python/tests/test_key_value.py
+++ b/modules/platforms/python/tests/test_key_value.py
@@ -13,8 +13,12 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+from datetime import datetime
+
 from pyignite.api import *
-from pyignite.datatypes import IntObject
+from pyignite.datatypes import (
+    CollectionObject, IntObject, MapObject, TimestampObject,
+)
 
 
 def test_put_get(client, cache):
@@ -325,3 +329,72 @@ def test_cache_get_size(client, cache):
     result = cache_get_size(client, cache)
     assert result.status == 0
     assert result.value == 1
+
+
+def test_put_get_collection(client):
+
+    test_datetime = datetime(year=1996, month=3, day=1)
+
+    cache = client.get_or_create_cache('test_coll_cache')
+    cache.put(
+        'simple',
+        (
+            1,
+            [
+                (123, IntObject),
+                678,
+                None,
+                55.2,
+                ((test_datetime, 0), TimestampObject),
+            ]
+        ),
+        value_hint=CollectionObject
+    )
+    value = cache.get('simple')
+    assert value == (1, [123, 678, None, 55.2, (test_datetime, 0)])
+
+    cache.put(
+        'nested',
+        (
+            1,
+            [
+                123,
+                ((1, [456, 'inner_test_string', 789]), CollectionObject),
+                'outer_test_string',
+            ]
+        ),
+        value_hint=CollectionObject
+    )
+    value = cache.get('nested')
+    assert value == (
+        1,
+        [
+            123,
+            (1, [456, 'inner_test_string', 789]),
+            'outer_test_string'
+        ]
+    )
+
+
+def test_put_get_map(client):
+
+    cache = client.get_or_create_cache('test_map_cache')
+
+    cache.put(
+        'test_map',
+        (
+            MapObject.HASH_MAP,
+            {
+                (123, IntObject): 'test_data',
+                456: ((1, [456, 'inner_test_string', 789]), CollectionObject),
+                'test_key': 32.4,
+            }
+        ),
+        value_hint=MapObject
+    )
+    value = cache.get('test_map')
+    assert value == (MapObject.HASH_MAP, {
+        123: 'test_data',
+        456: (1, [456, 'inner_test_string', 789]),
+        'test_key': 32.4,
+    })


[41/50] [abbrv] ignite git commit: IGNITE-10193 Removed non-actual muted baseline deletion test - Fixes #5446.

Posted by ag...@apache.org.
IGNITE-10193 Removed non-actual muted baseline deletion test - Fixes #5446.

Signed-off-by: Pavel Kovalenko <jo...@gmail.com>


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

Branch: refs/heads/ignite-9720
Commit: 566adaac9b70cbba94b97afadecb9525b3900733
Parents: be97bcd
Author: Alexey Platonov <ap...@gmail.com>
Authored: Tue Nov 27 20:30:42 2018 +0300
Committer: Pavel Kovalenko <jo...@gmail.com>
Committed: Tue Nov 27 20:30:42 2018 +0300

----------------------------------------------------------------------
 ...eBaselineAffinityTopologyActivationTest.java | 59 --------------------
 1 file changed, 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/566adaac/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteBaselineAffinityTopologyActivationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteBaselineAffinityTopologyActivationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteBaselineAffinityTopologyActivationTest.java
index f44e792..838c732 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteBaselineAffinityTopologyActivationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgniteBaselineAffinityTopologyActivationTest.java
@@ -1030,65 +1030,6 @@ public class IgniteBaselineAffinityTopologyActivationTest extends GridCommonAbst
     }
 
     /**
-     * Restore this test when requirements for BaselineTopology deletion are clarified and this feature
-     * is covered with more tests.
-     */
-    public void _testBaselineTopologyHistoryIsDeletedOnBaselineDelete() throws Exception {
-        BaselineTopologyHistoryVerifier verifier = new BaselineTopologyHistoryVerifier() {
-            @Override public void verify(BaselineTopologyHistory bltHist) {
-                assertNotNull(bltHist);
-
-                assertEquals(0, bltHist.history().size());
-            }
-        };
-
-        Ignite nodeA = startGridWithConsistentId("A");
-        startGridWithConsistentId("B");
-        startGridWithConsistentId("C");
-
-        nodeA.cluster().active(true);
-
-        stopAllGrids(false);
-
-        nodeA = startGridWithConsistentId("A");
-        startGridWithConsistentId("B");
-
-        nodeA.cluster().active(true);
-
-        nodeA.cluster().setBaselineTopology(baselineNodes(nodeA.cluster().forServers().nodes()));
-
-        stopAllGrids(false);
-
-        nodeA = startGridWithConsistentId("A");
-
-        nodeA.cluster().active(true);
-
-        nodeA.cluster().setBaselineTopology(baselineNodes(nodeA.cluster().forServers().nodes()));
-
-        stopAllGrids(false);
-
-        final Ignite node = startGridWithConsistentId("A");
-
-        boolean activated = GridTestUtils.waitForCondition(new GridAbsPredicate() {
-            @Override public boolean apply() {
-                return node.cluster().active();
-            }
-        }, 10_000);
-
-        assertTrue(activated);
-
-        node.cluster().setBaselineTopology(null);
-
-        verifyBaselineTopologyHistoryOnNodes(verifier, new Ignite[] {node});
-
-        stopAllGrids(false);
-
-        nodeA = startGridWithConsistentId("A");
-
-        verifyBaselineTopologyHistoryOnNodes(verifier, new Ignite[] {nodeA});
-    }
-
-    /**
      * Retrieves baseline topology from ignite node instance.
      *
      * @param ig Ig.


[12/50] [abbrv] ignite git commit: IGNITE-10216: disable sort annotations in inspection config - Fixes #5471.

Posted by ag...@apache.org.
IGNITE-10216: disable sort annotations in inspection config - Fixes #5471.

Signed-off-by: Nikolay Izhikov <ni...@apache.org>


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

Branch: refs/heads/ignite-9720
Commit: cdaeda108c7ce42e9b32264a37ce4ccbdbf55e42
Parents: 475a0a7
Author: Maxim Muzafarov <ma...@gmail.com>
Authored: Fri Nov 23 23:03:15 2018 +0300
Committer: Nikolay Izhikov <ni...@apache.org>
Committed: Fri Nov 23 23:03:15 2018 +0300

----------------------------------------------------------------------
 idea/ignite_inspections.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cdaeda10/idea/ignite_inspections.xml
----------------------------------------------------------------------
diff --git a/idea/ignite_inspections.xml b/idea/ignite_inspections.xml
index 7ce5af8..9695fca 100644
--- a/idea/ignite_inspections.xml
+++ b/idea/ignite_inspections.xml
@@ -518,7 +518,7 @@
     <option name="ignoreAnonymousClassMethods" value="false" />
   </inspection_tool>
   <inspection_tool class="MissortedModifiers" enabled="true" level="WARNING" enabled_by_default="true">
-    <option name="m_requireAnnotationsFirst" value="true" />
+    <option name="m_requireAnnotationsFirst" value="false" />
   </inspection_tool>
   <inspection_tool class="MisspelledCompareTo" enabled="true" level="WARNING" enabled_by_default="true" />
   <inspection_tool class="MisspelledEquals" enabled="true" level="WARNING" enabled_by_default="true" />


[05/50] [abbrv] ignite git commit: IGNITE-10335: move ML examples datasets files to resources

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/1dea0a42/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/titanic.csv
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/titanic.csv b/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/titanic.csv
deleted file mode 100644
index 6994016..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/ml/util/datasets/titanic.csv
+++ /dev/null
@@ -1,1310 +0,0 @@
-pclass;survived;name;sex;age;sibsp;parch;ticket;fare;cabin;embarked;boat;body;homedest
-1;1;Allen, Miss. Elisabeth Walton;;29;;;24160;211,3375;B5;;2;;St Louis, MO
-1;1;Allison, Master. Hudson Trevor;male;0,9167;1;2;113781;151,55;C22 C26;AA;11;;Montreal, PQ / Chesterville, ON
-1;0;Allison, Miss. Helen Loraine;female;2;1;2;113781;151,55;C22 C26;S;;;Montreal, PQ / Chesterville, ON
-1;0;Allison, Mr. Hudson Joshua Creighton;male;30;1;2;113781;151,55;C22 C26;S;;135;Montreal, PQ / Chesterville, ON
-1;0;Allison, Mrs. Hudson J C (Bessie Waldo Daniels);female;25;1;2;113781;151,55;C22 C26;S;;;Montreal, PQ / Chesterville, ON
-1;1;Anderson, Mr. Harry;male;48;0;0;19952;26,55;E12;S;3;;New York, NY
-1;1;Andrews, Miss. Kornelia Theodosia;female;63;1;0;13502;77,9583;D7;S;10;;Hudson, NY
-1;0;Andrews, Mr. Thomas Jr;male;39;0;0;112050;0;A36;S;;;Belfast, NI
-1;1;Appleton, Mrs. Edward Dale (Charlotte Lamson);female;53;2;0;11769;51,4792;C101;S;D;;Bayside, Queens, NY
-1;0;Artagaveytia, Mr. Ramon;male;71;0;0;PC 17609;49,5042;;C;;22;Montevideo, Uruguay
-1;0;Astor, Col. John Jacob;male;47;1;0;PC 17757;227,525;C62 C64;C;;124;New York, NY
-1;1;Astor, Mrs. John Jacob (Madeleine Talmadge Force);female;18;1;0;PC 17757;227,525;C62 C64;C;4;;New York, NY
-1;1;Aubart, Mme. Leontine Pauline;female;24;0;0;PC 17477;69,3;B35;C;9;;Paris, France
-1;1;"Barber, Miss. Ellen ""Nellie""";female;26;0;0;19877;78,85;;S;6;;
-1;1;Barkworth, Mr. Algernon Henry Wilson;male;80;0;0;27042;30;A23;S;B;;Hessle, Yorks
-1;0;Baumann, Mr. John D;male;;0;0;PC 17318;25,925;;S;;;New York, NY
-1;0;Baxter, Mr. Quigg Edmond;male;24;0;1;PC 17558;247,5208;B58 B60;C;;;Montreal, PQ
-1;1;Baxter, Mrs. James (Helene DeLaudeniere Chaput);female;50;0;1;PC 17558;247,5208;B58 B60;C;6;;Montreal, PQ
-1;1;Bazzani, Miss. Albina;female;32;0;0;11813;76,2917;D15;C;8;;
-1;0;Beattie, Mr. Thomson;male;36;0;0;13050;75,2417;C6;C;A;;Winnipeg, MN
-1;1;Beckwith, Mr. Richard Leonard;male;37;1;1;11751;52,5542;D35;S;5;;New York, NY
-1;1;Beckwith, Mrs. Richard Leonard (Sallie Monypeny);female;47;1;1;11751;52,5542;D35;S;5;;New York, NY
-1;1;Behr, Mr. Karl Howell;male;26;0;0;111369;30;C148;C;5;;New York, NY
-1;1;Bidois, Miss. Rosalie;female;42;0;0;PC 17757;227,525;;C;4;;
-1;1;Bird, Miss. Ellen;female;29;0;0;PC 17483;221,7792;C97;S;8;;
-1;0;Birnbaum, Mr. Jakob;male;25;0;0;13905;26;;C;;148;San Francisco, CA
-1;1;Bishop, Mr. Dickinson H;male;25;1;0;11967;91,0792;B49;C;7;;Dowagiac, MI
-1;1;Bishop, Mrs. Dickinson H (Helen Walton);female;19;1;0;11967;91,0792;B49;C;7;;Dowagiac, MI
-1;1;Bissette, Miss. Amelia;female;35;0;0;PC 17760;135,6333;C99;S;8;;
-1;1;Bjornstrom-Steffansson, Mr. Mauritz Hakan;male;28;0;0;110564;26,55;C52;S;D;;Stockholm, Sweden / Washington, DC
-1;0;Blackwell, Mr. Stephen Weart;male;45;0;0;113784;35,5;T;S;;;Trenton, NJ
-1;1;Blank, Mr. Henry;male;40;0;0;112277;31;A31;C;7;;Glen Ridge, NJ
-1;1;Bonnell, Miss. Caroline;female;30;0;0;36928;164,8667;C7;S;8;;Youngstown, OH
-1;1;Bonnell, Miss. Elizabeth;female;58;0;0;113783;26,55;C103;S;8;;Birkdale, England Cleveland, Ohio
-1;0;Borebank, Mr. John James;male;42;0;0;110489;26,55;D22;S;;;London / Winnipeg, MB
-1;1;Bowen, Miss. Grace Scott;female;45;0;0;PC 17608;262,375;;C;4;;Cooperstown, NY
-1;1;Bowerman, Miss. Elsie Edith;female;22;0;1;113505;55;E33;S;6;;St Leonards-on-Sea, England Ohio
-1;1;"Bradley, Mr. George (""George Arthur Brayton"")";male;;0;0;111427;26,55;;S;9;;Los Angeles, CA
-1;0;Brady, Mr. John Bertram;male;41;0;0;113054;30,5;A21;S;;;Pomeroy, WA
-1;0;Brandeis, Mr. Emil;male;48;0;0;PC 17591;50,4958;B10;C;;208;Omaha, NE
-1;0;Brewe, Dr. Arthur Jackson;male;;0;0;112379;39,6;;C;;;Philadelphia, PA
-1;1;Brown, Mrs. James Joseph (Margaret Tobin);female;44;0;0;PC 17610;27,7208;B4;C;6;;Denver, CO
-1;1;Brown, Mrs. John Murray (Caroline Lane Lamson);female;59;2;0;11769;51,4792;C101;S;D;;Belmont, MA
-1;1;Bucknell, Mrs. William Robert (Emma Eliza Ward);female;60;0;0;11813;76,2917;D15;C;8;;Philadelphia, PA
-1;1;Burns, Miss. Elizabeth Margaret;female;41;0;0;16966;134,5;E40;C;3;;
-1;0;Butt, Major. Archibald Willingham;male;45;0;0;113050;26,55;B38;S;;;Washington, DC
-1;0;Cairns, Mr. Alexander;male;;0;0;113798;31;;S;;;
-1;1;Calderhead, Mr. Edward Pennington;male;42;0;0;PC 17476;26,2875;E24;S;5;;New York, NY
-1;1;Candee, Mrs. Edward (Helen Churchill Hungerford);female;53;0;0;PC 17606;27,4458;;C;6;;Washington, DC
-1;1;Cardeza, Mr. Thomas Drake Martinez;male;36;0;1;PC 17755;512,3292;B51 B53 B55;C;3;;Austria-Hungary / Germantown, Philadelphia, PA
-1;1;Cardeza, Mrs. James Warburton Martinez (Charlotte Wardle Drake);female;58;0;1;PC 17755;512,3292;B51 B53 B55;C;3;;Germantown, Philadelphia, PA
-1;0;Carlsson, Mr. Frans Olof;male;33;0;0;695;5;B51 B53 B55;S;;;New York, NY
-1;0;Carrau, Mr. Francisco M;male;28;0;0;113059;47,1;;S;;;Montevideo, Uruguay
-1;0;Carrau, Mr. Jose Pedro;male;17;0;0;113059;47,1;;S;;;Montevideo, Uruguay
-1;1;Carter, Master. William Thornton II;male;11;1;2;113760;120;B96 B98;S;4;;Bryn Mawr, PA
-1;1;Carter, Miss. Lucile Polk;female;14;1;2;113760;120;B96 B98;S;4;;Bryn Mawr, PA
-1;1;Carter, Mr. William Ernest;male;36;1;2;113760;120;B96 B98;S;C;;Bryn Mawr, PA
-1;1;Carter, Mrs. William Ernest (Lucile Polk);female;36;1;2;113760;120;B96 B98;S;4;;Bryn Mawr, PA
-1;0;Case, Mr. Howard Brown;male;49;0;0;19924;26;;S;;;Ascot, Berkshire / Rochester, NY
-1;1;Cassebeer, Mrs. Henry Arthur Jr (Eleanor Genevieve Fosdick);female;;0;0;17770;27,7208;;C;5;;New York, NY
-1;0;Cavendish, Mr. Tyrell William;male;36;1;0;19877;78,85;C46;S;;172;Little Onn Hall, Staffs
-1;1;Cavendish, Mrs. Tyrell William (Julia Florence Siegel);female;76;1;0;19877;78,85;C46;S;6;;Little Onn Hall, Staffs
-1;0;Chaffee, Mr. Herbert Fuller;male;46;1;0;W.E.P. 5734;61,175;E31;S;;;Amenia, ND
-1;1;Chaffee, Mrs. Herbert Fuller (Carrie Constance Toogood);female;47;1;0;W.E.P. 5734;61,175;E31;S;4;;Amenia, ND
-1;1;Chambers, Mr. Norman Campbell;male;27;1;0;113806;53,1;E8;S;5;;New York, NY / Ithaca, NY
-1;1;Chambers, Mrs. Norman Campbell (Bertha Griggs);female;33;1;0;113806;53,1;E8;S;5;;New York, NY / Ithaca, NY
-1;1;Chaudanson, Miss. Victorine;female;36;0;0;PC 17608;262,375;B61;C;4;;
-1;1;Cherry, Miss. Gladys;female;30;0;0;110152;86,5;B77;S;8;;London, England
-1;1;Chevre, Mr. Paul Romaine;male;45;0;0;PC 17594;29,7;A9;C;7;;Paris, France
-1;1;Chibnall, Mrs. (Edith Martha Bowerman);female;;0;1;113505;55;E33;S;6;;St Leonards-on-Sea, England Ohio
-1;0;Chisholm, Mr. Roderick Robert Crispin;male;;0;0;112051;0;;S;;;Liverpool, England / Belfast
-1;0;Clark, Mr. Walter Miller;male;27;1;0;13508;136,7792;C89;C;;;Los Angeles, CA
-1;1;Clark, Mrs. Walter Miller (Virginia McDowell);female;26;1;0;13508;136,7792;C89;C;4;;Los Angeles, CA
-1;1;Cleaver, Miss. Alice;female;22;0;0;113781;151,55;;S;11;;
-1;0;Clifford, Mr. George Quincy;male;;0;0;110465;52;A14;S;;;Stoughton, MA
-1;0;Colley, Mr. Edward Pomeroy;male;47;0;0;5727;25,5875;E58;S;;;Victoria, BC
-1;1;Compton, Miss. Sara Rebecca;female;39;1;1;PC 17756;83,1583;E49;C;14;;Lakewood, NJ
-1;0;Compton, Mr. Alexander Taylor Jr;male;37;1;1;PC 17756;83,1583;E52;C;;;Lakewood, NJ
-1;1;Compton, Mrs. Alexander Taylor (Mary Eliza Ingersoll);female;64;0;2;PC 17756;83,1583;E45;C;14;;Lakewood, NJ
-1;1;Cornell, Mrs. Robert Clifford (Malvina Helen Lamson);female;55;2;0;11770;25,7;C101;S;2;;New York, NY
-1;0;Crafton, Mr. John Bertram;male;;0;0;113791;26,55;;S;;;Roachdale, IN
-1;0;Crosby, Capt. Edward Gifford;male;70;1;1;WE/P 5735;71;B22;S;;269;Milwaukee, WI
-1;1;Crosby, Miss. Harriet R;female;36;0;2;WE/P 5735;71;B22;S;7;;Milwaukee, WI
-1;1;Crosby, Mrs. Edward Gifford (Catherine Elizabeth Halstead);female;64;1;1;112901;26,55;B26;S;7;;Milwaukee, WI
-1;0;Cumings, Mr. John Bradley;male;39;1;0;PC 17599;71,2833;C85;C;;;New York, NY
-1;1;Cumings, Mrs. John Bradley (Florence Briggs Thayer);female;38;1;0;PC 17599;71,2833;C85;C;4;;New York, NY
-1;1;Daly, Mr. Peter Denis ;male;51;0;0;113055;26,55;E17;S;5 9;;Lima, Peru
-1;1;Daniel, Mr. Robert Williams;male;27;0;0;113804;30,5;;S;3;;Philadelphia, PA
-1;1;Daniels, Miss. Sarah;female;33;0;0;113781;151,55;;S;8;;
-1;0;Davidson, Mr. Thornton;male;31;1;0;F.C. 12750;52;B71;S;;;Montreal, PQ
-1;1;Davidson, Mrs. Thornton (Orian Hays);female;27;1;2;F.C. 12750;52;B71;S;3;;Montreal, PQ
-1;1;Dick, Mr. Albert Adrian;male;31;1;0;17474;57;B20;S;3;;Calgary, AB
-1;1;Dick, Mrs. Albert Adrian (Vera Gillespie);female;17;1;0;17474;57;B20;S;3;;Calgary, AB
-1;1;Dodge, Dr. Washington;male;53;1;1;33638;81,8583;A34;S;13;;San Francisco, CA
-1;1;Dodge, Master. Washington;male;4;0;2;33638;81,8583;A34;S;5;;San Francisco, CA
-1;1;Dodge, Mrs. Washington (Ruth Vidaver);female;54;1;1;33638;81,8583;A34;S;5;;San Francisco, CA
-1;0;Douglas, Mr. Walter Donald;male;50;1;0;PC 17761;106,425;C86;C;;62;Deephaven, MN / Cedar Rapids, IA
-1;1;Douglas, Mrs. Frederick Charles (Mary Helene Baxter);female;27;1;1;PC 17558;247,5208;B58 B60;C;6;;Montreal, PQ
-1;1;Douglas, Mrs. Walter Donald (Mahala Dutton);female;48;1;0;PC 17761;106,425;C86;C;2;;Deephaven, MN / Cedar Rapids, IA
-1;1;"Duff Gordon, Lady. (Lucille Christiana Sutherland) (""Mrs Morgan"")";female;48;1;0;11755;39,6;A16;C;1;;London / Paris
-1;1;"Duff Gordon, Sir. Cosmo Edmund (""Mr Morgan"")";male;49;1;0;PC 17485;56,9292;A20;C;1;;London / Paris
-1;0;Dulles, Mr. William Crothers;male;39;0;0;PC 17580;29,7;A18;C;;133;Philadelphia, PA
-1;1;Earnshaw, Mrs. Boulton (Olive Potter);female;23;0;1;11767;83,1583;C54;C;7;;Mt Airy, Philadelphia, PA
-1;1;Endres, Miss. Caroline Louise;female;38;0;0;PC 17757;227,525;C45;C;4;;New York, NY
-1;1;Eustis, Miss. Elizabeth Mussey;female;54;1;0;36947;78,2667;D20;C;4;;Brookline, MA
-1;0;Evans, Miss. Edith Corse;female;36;0;0;PC 17531;31,6792;A29;C;;;New York, NY
-1;0;Farthing, Mr. John;male;;0;0;PC 17483;221,7792;C95;S;;;
-1;1;Flegenheim, Mrs. Alfred (Antoinette);female;;0;0;PC 17598;31,6833;;S;7;;New York, NY
-1;1;Fleming, Miss. Margaret;female;;0;0;17421;110,8833;;C;4;;
-1;1;"Flynn, Mr. John Irwin (""Irving"")";male;36;0;0;PC 17474;26,3875;E25;S;5;;Brooklyn, NY
-1;0;Foreman, Mr. Benjamin Laventall;male;30;0;0;113051;27,75;C111;C;;;New York, NY
-1;1;Fortune, Miss. Alice Elizabeth;female;24;3;2;19950;263;C23 C25 C27;S;10;;Winnipeg, MB
-1;1;Fortune, Miss. Ethel Flora;female;28;3;2;19950;263;C23 C25 C27;S;10;;Winnipeg, MB
-1;1;Fortune, Miss. Mabel Helen;female;23;3;2;19950;263;C23 C25 C27;S;10;;Winnipeg, MB
-1;0;Fortune, Mr. Charles Alexander;male;19;3;2;19950;263;C23 C25 C27;S;;;Winnipeg, MB
-1;0;Fortune, Mr. Mark;male;64;1;4;19950;263;C23 C25 C27;S;;;Winnipeg, MB
-1;1;Fortune, Mrs. Mark (Mary McDougald);female;60;1;4;19950;263;C23 C25 C27;S;10;;Winnipeg, MB
-1;1;Francatelli, Miss. Laura Mabel;female;30;0;0;PC 17485;56,9292;E36;C;1;;
-1;0;Franklin, Mr. Thomas Parham;male;;0;0;113778;26,55;D34;S;;;Westcliff-on-Sea, Essex
-1;1;Frauenthal, Dr. Henry William;male;50;2;0;PC 17611;133,65;;S;5;;New York, NY
-1;1;Frauenthal, Mr. Isaac Gerald;male;43;1;0;17765;27,7208;D40;C;5;;New York, NY
-1;1;Frauenthal, Mrs. Henry William (Clara Heinsheimer);female;;1;0;PC 17611;133,65;;S;5;;New York, NY
-1;1;Frolicher, Miss. Hedwig Margaritha;female;22;0;2;13568;49,5;B39;C;5;;Zurich, Switzerland
-1;1;Frolicher-Stehli, Mr. Maxmillian;male;60;1;1;13567;79,2;B41;C;5;;Zurich, Switzerland
-1;1;Frolicher-Stehli, Mrs. Maxmillian (Margaretha Emerentia Stehli);female;48;1;1;13567;79,2;B41;C;5;;Zurich, Switzerland
-1;0;Fry, Mr. Richard;male;;0;0;112058;0;B102;S;;;
-1;0;Futrelle, Mr. Jacques Heath;male;37;1;0;113803;53,1;C123;S;;;Scituate, MA
-1;1;Futrelle, Mrs. Jacques Heath (Lily May Peel);female;35;1;0;113803;53,1;C123;S;D;;Scituate, MA
-1;0;Gee, Mr. Arthur H;male;47;0;0;111320;38,5;E63;S;;275;St Anne's-on-Sea, Lancashire
-1;1;Geiger, Miss. Amalie;female;35;0;0;113503;211,5;C130;C;4;;
-1;1;Gibson, Miss. Dorothy Winifred;female;22;0;1;112378;59,4;;C;7;;New York, NY
-1;1;Gibson, Mrs. Leonard (Pauline C Boeson);female;45;0;1;112378;59,4;;C;7;;New York, NY
-1;0;Giglio, Mr. Victor;male;24;0;0;PC 17593;79,2;B86;C;;;
-1;1;Goldenberg, Mr. Samuel L;male;49;1;0;17453;89,1042;C92;C;5;;Paris, France / New York, NY
-1;1;Goldenberg, Mrs. Samuel L (Edwiga Grabowska);female;;1;0;17453;89,1042;C92;C;5;;Paris, France / New York, NY
-1;0;Goldschmidt, Mr. George B;male;71;0;0;PC 17754;34,6542;A5;C;;;New York, NY
-1;1;Gracie, Col. Archibald IV;male;53;0;0;113780;28,5;C51;C;B;;Washington, DC
-1;1;Graham, Miss. Margaret Edith;female;19;0;0;112053;30;B42;S;3;;Greenwich, CT
-1;0;Graham, Mr. George Edward;male;38;0;1;PC 17582;153,4625;C91;S;;147;Winnipeg, MB
-1;1;Graham, Mrs. William Thompson (Edith Junkins);female;58;0;1;PC 17582;153,4625;C125;S;3;;Greenwich, CT
-1;1;Greenfield, Mr. William Bertram;male;23;0;1;PC 17759;63,3583;D10 D12;C;7;;New York, NY
-1;1;Greenfield, Mrs. Leo David (Blanche Strouse);female;45;0;1;PC 17759;63,3583;D10 D12;C;7;;New York, NY
-1;0;Guggenheim, Mr. Benjamin;male;46;0;0;PC 17593;79,2;B82 B84;C;;;New York, NY
-1;1;Harder, Mr. George Achilles;male;25;1;0;11765;55,4417;E50;C;5;;Brooklyn, NY
-1;1;Harder, Mrs. George Achilles (Dorothy Annan);female;25;1;0;11765;55,4417;E50;C;5;;Brooklyn, NY
-1;1;Harper, Mr. Henry Sleeper;male;48;1;0;PC 17572;76,7292;D33;C;3;;New York, NY
-1;1;Harper, Mrs. Henry Sleeper (Myna Haxtun);female;49;1;0;PC 17572;76,7292;D33;C;3;;New York, NY
-1;0;Harrington, Mr. Charles H;male;;0;0;113796;42,4;;S;;;
-1;0;Harris, Mr. Henry Birkhardt;male;45;1;0;36973;83,475;C83;S;;;New York, NY
-1;1;Harris, Mrs. Henry Birkhardt (Irene Wallach);female;35;1;0;36973;83,475;C83;S;D;;New York, NY
-1;0;Harrison, Mr. William;male;40;0;0;112059;0;B94;S;;110;
-1;1;Hassab, Mr. Hammad;male;27;0;0;PC 17572;76,7292;D49;C;3;;
-1;1;Hawksford, Mr. Walter James;male;;0;0;16988;30;D45;S;3;;Kingston, Surrey
-1;1;Hays, Miss. Margaret Bechstein;female;24;0;0;11767;83,1583;C54;C;7;;New York, NY
-1;0;Hays, Mr. Charles Melville;male;55;1;1;12749;93,5;B69;S;;307;Montreal, PQ
-1;1;Hays, Mrs. Charles Melville (Clara Jennings Gregg);female;52;1;1;12749;93,5;B69;S;3;;Montreal, PQ
-1;0;Head, Mr. Christopher;male;42;0;0;113038;42,5;B11;S;;;London / Middlesex
-1;0;Hilliard, Mr. Herbert Henry;male;;0;0;17463;51,8625;E46;S;;;Brighton, MA
-1;0;Hipkins, Mr. William Edward;male;55;0;0;680;50;C39;S;;;London / Birmingham
-1;1;Hippach, Miss. Jean Gertrude;female;16;0;1;111361;57,9792;B18;C;4;;Chicago, IL
-1;1;Hippach, Mrs. Louis Albert (Ida Sophia Fischer);female;44;0;1;111361;57,9792;B18;C;4;;Chicago, IL
-1;1;Hogeboom, Mrs. John C (Anna Andrews);female;51;1;0;13502;77,9583;D11;S;10;;Hudson, NY
-1;0;Holverson, Mr. Alexander Oskar;male;42;1;0;113789;52;;S;;38;New York, NY
-1;1;Holverson, Mrs. Alexander Oskar (Mary Aline Towner);female;35;1;0;113789;52;;S;8;;New York, NY
-1;1;"Homer, Mr. Harry (""Mr E Haven"")";male;35;0;0;111426;26,55;;C;15;;Indianapolis, IN
-1;1;Hoyt, Mr. Frederick Maxfield;male;38;1;0;19943;90;C93;S;D;;New York, NY /  Stamford CT
-1;0;Hoyt, Mr. William Fisher;male;;0;0;PC 17600;30,6958;;C;14;;New York, NY
-1;1;Hoyt, Mrs. Frederick Maxfield (Jane Anne Forby);female;35;1;0;19943;90;C93;S;D;;New York, NY /  Stamford CT
-1;1;Icard, Miss. Amelie;female;38;0;0;113572;80;B28;;6;;
-1;0;Isham, Miss. Ann Elizabeth;female;50;0;0;PC 17595;28,7125;C49;C;;;Paris, France New York, NY
-1;1;Ismay, Mr. Joseph Bruce;male;49;0;0;112058;0;B52 B54 B56;S;C;;Liverpool
-1;0;Jones, Mr. Charles Cresson;male;46;0;0;694;26;;S;;80;Bennington, VT
-1;0;Julian, Mr. Henry Forbes;male;50;0;0;113044;26;E60;S;;;London
-1;0;Keeping, Mr. Edwin;male;32,5;0;0;113503;211,5;C132;C;;45;
-1;0;Kent, Mr. Edward Austin;male;58;0;0;11771;29,7;B37;C;;258;Buffalo, NY
-1;0;Kenyon, Mr. Frederick R;male;41;1;0;17464;51,8625;D21;S;;;Southington / Noank, CT
-1;1;Kenyon, Mrs. Frederick R (Marion);female;;1;0;17464;51,8625;D21;S;8;;Southington / Noank, CT
-1;1;Kimball, Mr. Edwin Nelson Jr;male;42;1;0;11753;52,5542;D19;S;5;;Boston, MA
-1;1;Kimball, Mrs. Edwin Nelson Jr (Gertrude Parsons);female;45;1;0;11753;52,5542;D19;S;5;;Boston, MA
-1;0;Klaber, Mr. Herman;male;;0;0;113028;26,55;C124;S;;;Portland, OR
-1;1;Kreuchen, Miss. Emilie;female;39;0;0;24160;211,3375;;S;2;;
-1;1;Leader, Dr. Alice (Farnham);female;49;0;0;17465;25,9292;D17;S;8;;New York, NY
-1;1;LeRoy, Miss. Bertha;female;30;0;0;PC 17761;106,425;;C;2;;
-1;1;Lesurer, Mr. Gustave J;male;35;0;0;PC 17755;512,3292;B101;C;3;;
-1;0;Lewy, Mr. Ervin G;male;;0;0;PC 17612;27,7208;;C;;;Chicago, IL
-1;0;"Lindeberg-Lind, Mr. Erik Gustaf (""Mr Edward Lingrey"")";male;42;0;0;17475;26,55;;S;;;Stockholm, Sweden
-1;1;Lindstrom, Mrs. Carl Johan (Sigrid Posse);female;55;0;0;112377;27,7208;;C;6;;Stockholm, Sweden
-1;1;Lines, Miss. Mary Conover;female;16;0;1;PC 17592;39,4;D28;S;9;;Paris, France
-1;1;Lines, Mrs. Ernest H (Elizabeth Lindsey James);female;51;0;1;PC 17592;39,4;D28;S;9;;Paris, France
-1;0;Long, Mr. Milton Clyde;male;29;0;0;113501;30;D6;S;;126;Springfield, MA
-1;1;Longley, Miss. Gretchen Fiske;female;21;0;0;13502;77,9583;D9;S;10;;Hudson, NY
-1;0;Loring, Mr. Joseph Holland;male;30;0;0;113801;45,5;;S;;;London / New York, NY
-1;1;Lurette, Miss. Elise;female;58;0;0;PC 17569;146,5208;B80;C;;;
-1;1;Madill, Miss. Georgette Alexandra;female;15;0;1;24160;211,3375;B5;S;2;;St Louis, MO
-1;0;Maguire, Mr. John Edward;male;30;0;0;110469;26;C106;S;;;Brockton, MA
-1;1;Maioni, Miss. Roberta;female;16;0;0;110152;86,5;B79;S;8;;
-1;1;Marechal, Mr. Pierre;male;;0;0;11774;29,7;C47;C;7;;Paris, France
-1;0;Marvin, Mr. Daniel Warner;male;19;1;0;113773;53,1;D30;S;;;New York, NY
-1;1;Marvin, Mrs. Daniel Warner (Mary Graham Carmichael Farquarson);female;18;1;0;113773;53,1;D30;S;10;;New York, NY
-1;1;"Mayne, Mlle. Berthe Antonine (""Mrs de Villiers"")";female;24;0;0;PC 17482;49,5042;C90;C;6;;Belgium  Montreal, PQ
-1;0;McCaffry, Mr. Thomas Francis;male;46;0;0;13050;75,2417;C6;C;;292;Vancouver, BC
-1;0;McCarthy, Mr. Timothy J;male;54;0;0;17463;51,8625;E46;S;;175;Dorchester, MA
-1;1;McGough, Mr. James Robert;male;36;0;0;PC 17473;26,2875;E25;S;7;;Philadelphia, PA
-1;0;Meyer, Mr. Edgar Joseph;male;28;1;0;PC 17604;82,1708;;C;;;New York, NY
-1;1;Meyer, Mrs. Edgar Joseph (Leila Saks);female;;1;0;PC 17604;82,1708;;C;6;;New York, NY
-1;0;Millet, Mr. Francis Davis;male;65;0;0;13509;26,55;E38;S;;249;East Bridgewater, MA
-1;0;Minahan, Dr. William Edward;male;44;2;0;19928;90;C78;Q;;230;Fond du Lac, WI
-1;1;Minahan, Miss. Daisy E;female;33;1;0;19928;90;C78;Q;14;;Green Bay, WI
-1;1;Minahan, Mrs. William Edward (Lillian E Thorpe);female;37;1;0;19928;90;C78;Q;14;;Fond du Lac, WI
-1;1;Mock, Mr. Philipp Edmund;male;30;1;0;13236;57,75;C78;C;11;;New York, NY
-1;0;Molson, Mr. Harry Markland;male;55;0;0;113787;30,5;C30;S;;;Montreal, PQ
-1;0;Moore, Mr. Clarence Bloomfield;male;47;0;0;113796;42,4;;S;;;Washington, DC
-1;0;Natsch, Mr. Charles H;male;37;0;1;PC 17596;29,7;C118;C;;;Brooklyn, NY
-1;1;Newell, Miss. Madeleine;female;31;1;0;35273;113,275;D36;C;6;;Lexington, MA
-1;1;Newell, Miss. Marjorie;female;23;1;0;35273;113,275;D36;C;6;;Lexington, MA
-1;0;Newell, Mr. Arthur Webster;male;58;0;2;35273;113,275;D48;C;;122;Lexington, MA
-1;1;Newsom, Miss. Helen Monypeny;female;19;0;2;11752;26,2833;D47;S;5;;New York, NY
-1;0;Nicholson, Mr. Arthur Ernest;male;64;0;0;693;26;;S;;263;Isle of Wight, England
-1;1;Oliva y Ocana, Dona. Fermina;female;39;0;0;PC 17758;108,9;C105;C;8;;
-1;1;Omont, Mr. Alfred Fernand;male;;0;0;F.C. 12998;25,7417;;C;7;;Paris, France
-1;1;Ostby, Miss. Helene Ragnhild;female;22;0;1;113509;61,9792;B36;C;5;;Providence, RI
-1;0;Ostby, Mr. Engelhart Cornelius;male;65;0;1;113509;61,9792;B30;C;;234;Providence, RI
-1;0;Ovies y Rodriguez, Mr. Servando;male;28,5;0;0;PC 17562;27,7208;D43;C;;189;?Havana, Cuba
-1;0;Parr, Mr. William Henry Marsh;male;;0;0;112052;0;;S;;;Belfast
-1;0;Partner, Mr. Austen;male;45,5;0;0;113043;28,5;C124;S;;166;Surbiton Hill, Surrey
-1;0;Payne, Mr. Vivian Ponsonby;male;23;0;0;12749;93,5;B24;S;;;Montreal, PQ
-1;0;Pears, Mr. Thomas Clinton;male;29;1;0;113776;66,6;C2;S;;;Isleworth, England
-1;1;Pears, Mrs. Thomas (Edith Wearne);female;22;1;0;113776;66,6;C2;S;8;;Isleworth, England
-1;0;Penasco y Castellana, Mr. Victor de Satode;male;18;1;0;PC 17758;108,9;C65;C;;;Madrid, Spain
-1;1;Penasco y Castellana, Mrs. Victor de Satode (Maria Josefa Perez de Soto y Vallejo);female;17;1;0;PC 17758;108,9;C65;C;8;;Madrid, Spain
-1;1;Perreault, Miss. Anne;female;30;0;0;12749;93,5;B73;S;3;;
-1;1;Peuchen, Major. Arthur Godfrey;male;52;0;0;113786;30,5;C104;S;6;;Toronto, ON
-1;0;Porter, Mr. Walter Chamberlain;male;47;0;0;110465;52;C110;S;;207;Worcester, MA
-1;1;Potter, Mrs. Thomas Jr (Lily Alexenia Wilson);female;56;0;1;11767;83,1583;C50;C;7;;Mt Airy, Philadelphia, PA
-1;0;Reuchlin, Jonkheer. John George;male;38;0;0;19972;0;;S;;;Rotterdam, Netherlands
-1;1;Rheims, Mr. George Alexander Lucien;male;;0;0;PC 17607;39,6;;S;A;;Paris /  New York, NY
-1;0;Ringhini, Mr. Sante;male;22;0;0;PC 17760;135,6333;;C;;232;
-1;0;Robbins, Mr. Victor;male;;0;0;PC 17757;227,525;;C;;;
-1;1;Robert, Mrs. Edward Scott (Elisabeth Walton McMillan);female;43;0;1;24160;211,3375;B3;S;2;;St Louis, MO
-1;0;Roebling, Mr. Washington Augustus II;male;31;0;0;PC 17590;50,4958;A24;S;;;Trenton, NJ
-1;1;"Romaine, Mr. Charles Hallace (""Mr C Rolmane"")";male;45;0;0;111428;26,55;;S;9;;New York, NY
-1;0;Rood, Mr. Hugh Roscoe;male;;0;0;113767;50;A32;S;;;Seattle, WA
-1;1;Rosenbaum, Miss. Edith Louise;female;33;0;0;PC 17613;27,7208;A11;C;11;;Paris, France
-1;0;"Rosenshine, Mr. George (""Mr George Thorne"")";male;46;0;0;PC 17585;79,2;;C;;16;New York, NY
-1;0;Ross, Mr. John Hugo;male;36;0;0;13049;40,125;A10;C;;;Winnipeg, MB
-1;1;Rothes, the Countess. of (Lucy Noel Martha Dyer-Edwards);female;33;0;0;110152;86,5;B77;S;8;;London  Vancouver, BC
-1;0;Rothschild, Mr. Martin;male;55;1;0;PC 17603;59,4;;C;;;New York, NY
-1;1;Rothschild, Mrs. Martin (Elizabeth L. Barrett);female;54;1;0;PC 17603;59,4;;C;6;;New York, NY
-1;0;Rowe, Mr. Alfred G;male;33;0;0;113790;26,55;;S;;109;London
-1;1;Ryerson, Master. John Borie;male;13;2;2;PC 17608;262,375;B57 B59 B63 B66;C;4;;Haverford, PA / Cooperstown, NY
-1;1;Ryerson, Miss. Emily Borie;female;18;2;2;PC 17608;262,375;B57 B59 B63 B66;C;4;;Haverford, PA / Cooperstown, NY
-1;1;"Ryerson, Miss. Susan Parker ""Suzette""";female;21;2;2;PC 17608;262,375;B57 B59 B63 B66;C;4;;Haverford, PA / Cooperstown, NY
-1;0;Ryerson, Mr. Arthur Larned;male;61;1;3;PC 17608;262,375;B57 B59 B63 B66;C;;;Haverford, PA / Cooperstown, NY
-1;1;Ryerson, Mrs. Arthur Larned (Emily Maria Borie);female;48;1;3;PC 17608;262,375;B57 B59 B63 B66;C;4;;Haverford, PA / Cooperstown, NY
-1;1;Saalfeld, Mr. Adolphe;male;;0;0;19988;30,5;C106;S;3;;Manchester, England
-1;1;Sagesser, Mlle. Emma;female;24;0;0;PC 17477;69,3;B35;C;9;;
-1;1;Salomon, Mr. Abraham L;male;;0;0;111163;26;;S;1;;New York, NY
-1;1;Schabert, Mrs. Paul (Emma Mock);female;35;1;0;13236;57,75;C28;C;11;;New York, NY
-1;1;Serepeca, Miss. Augusta;female;30;0;0;113798;31;;C;4;;
-1;1;Seward, Mr. Frederic Kimber;male;34;0;0;113794;26,55;;S;7;;New York, NY
-1;1;Shutes, Miss. Elizabeth W;female;40;0;0;PC 17582;153,4625;C125;S;3;;New York, NY / Greenwich CT
-1;1;Silverthorne, Mr. Spencer Victor;male;35;0;0;PC 17475;26,2875;E24;S;5;;St Louis, MO
-1;0;Silvey, Mr. William Baird;male;50;1;0;13507;55,9;E44;S;;;Duluth, MN
-1;1;Silvey, Mrs. William Baird (Alice Munger);female;39;1;0;13507;55,9;E44;S;11;;Duluth, MN
-1;1;Simonius-Blumer, Col. Oberst Alfons;male;56;0;0;13213;35,5;A26;C;3;;Basel, Switzerland
-1;1;Sloper, Mr. William Thompson;male;28;0;0;113788;35,5;A6;S;7;;New Britain, CT
-1;0;Smart, Mr. John Montgomery;male;56;0;0;113792;26,55;;S;;;New York, NY
-1;0;Smith, Mr. James Clinch;male;56;0;0;17764;30,6958;A7;C;;;St James, Long Island, NY
-1;0;Smith, Mr. Lucien Philip;male;24;1;0;13695;60;C31;S;;;Huntington, WV
-1;0;Smith, Mr. Richard William;male;;0;0;113056;26;A19;S;;;Streatham, Surrey
-1;1;Smith, Mrs. Lucien Philip (Mary Eloise Hughes);female;18;1;0;13695;60;C31;S;6;;Huntington, WV
-1;1;Snyder, Mr. John Pillsbury;male;24;1;0;21228;82,2667;B45;S;7;;Minneapolis, MN
-1;1;Snyder, Mrs. John Pillsbury (Nelle Stevenson);female;23;1;0;21228;82,2667;B45;S;7;;Minneapolis, MN
-1;1;Spedden, Master. Robert Douglas;male;6;0;2;16966;134,5;E34;C;3;;Tuxedo Park, NY
-1;1;Spedden, Mr. Frederic Oakley;male;45;1;1;16966;134,5;E34;C;3;;Tuxedo Park, NY
-1;1;Spedden, Mrs. Frederic Oakley (Margaretta Corning Stone);female;40;1;1;16966;134,5;E34;C;3;;Tuxedo Park, NY
-1;0;Spencer, Mr. William Augustus;male;57;1;0;PC 17569;146,5208;B78;C;;;Paris, France
-1;1;Spencer, Mrs. William Augustus (Marie Eugenie);female;;1;0;PC 17569;146,5208;B78;C;6;;Paris, France
-1;1;Stahelin-Maeglin, Dr. Max;male;32;0;0;13214;30,5;B50;C;3;;Basel, Switzerland
-1;0;Stead, Mr. William Thomas;male;62;0;0;113514;26,55;C87;S;;;Wimbledon Park, London / Hayling Island, Hants
-1;1;Stengel, Mr. Charles Emil Henry;male;54;1;0;11778;55,4417;C116;C;1;;Newark, NJ
-1;1;Stengel, Mrs. Charles Emil Henry (Annie May Morris);female;43;1;0;11778;55,4417;C116;C;5;;Newark, NJ
-1;1;Stephenson, Mrs. Walter Bertram (Martha Eustis);female;52;1;0;36947;78,2667;D20;C;4;;Haverford, PA
-1;0;Stewart, Mr. Albert A;male;;0;0;PC 17605;27,7208;;C;;;Gallipolis, Ohio / ? Paris / New York
-1;1;Stone, Mrs. George Nelson (Martha Evelyn);female;62;0;0;113572;80;B28;;6;;Cincinatti, OH
-1;0;Straus, Mr. Isidor;male;67;1;0;PC 17483;221,7792;C55 C57;S;;96;New York, NY
-1;0;Straus, Mrs. Isidor (Rosalie Ida Blun);female;63;1;0;PC 17483;221,7792;C55 C57;S;;;New York, NY
-1;0;Sutton, Mr. Frederick;male;61;0;0;36963;32,3208;D50;S;;46;Haddenfield, NJ
-1;1;Swift, Mrs. Frederick Joel (Margaret Welles Barron);female;48;0;0;17466;25,9292;D17;S;8;;Brooklyn, NY
-1;1;Taussig, Miss. Ruth;female;18;0;2;110413;79,65;E68;S;8;;New York, NY
-1;0;Taussig, Mr. Emil;male;52;1;1;110413;79,65;E67;S;;;New York, NY
-1;1;Taussig, Mrs. Emil (Tillie Mandelbaum);female;39;1;1;110413;79,65;E67;S;8;;New York, NY
-1;1;Taylor, Mr. Elmer Zebley;male;48;1;0;19996;52;C126;S;5 7;;London /  East Orange, NJ
-1;1;Taylor, Mrs. Elmer Zebley (Juliet Cummins Wright);female;;1;0;19996;52;C126;S;5 7;;London /  East Orange, NJ
-1;0;Thayer, Mr. John Borland;male;49;1;1;17421;110,8833;C68;C;;;Haverford, PA
-1;1;Thayer, Mr. John Borland Jr;male;17;0;2;17421;110,8833;C70;C;B;;Haverford, PA
-1;1;Thayer, Mrs. John Borland (Marian Longstreth Morris);female;39;1;1;17421;110,8833;C68;C;4;;Haverford, PA
-1;1;Thorne, Mrs. Gertrude Maybelle;female;;0;0;PC 17585;79,2;;C;D;;New York, NY
-1;1;Tucker, Mr. Gilbert Milligan Jr;male;31;0;0;2543;28,5375;C53;C;7;;Albany, NY
-1;0;Uruchurtu, Don. Manuel E;male;40;0;0;PC 17601;27,7208;;C;;;Mexico City, Mexico
-1;0;Van der hoef, Mr. Wyckoff;male;61;0;0;111240;33,5;B19;S;;245;Brooklyn, NY
-1;0;Walker, Mr. William Anderson;male;47;0;0;36967;34,0208;D46;S;;;East Orange, NJ
-1;1;Ward, Miss. Anna;female;35;0;0;PC 17755;512,3292;;C;3;;
-1;0;Warren, Mr. Frank Manley;male;64;1;0;110813;75,25;D37;C;;;Portland, OR
-1;1;Warren, Mrs. Frank Manley (Anna Sophia Atkinson);female;60;1;0;110813;75,25;D37;C;5;;Portland, OR
-1;0;Weir, Col. John;male;60;0;0;113800;26,55;;S;;;England Salt Lake City, Utah
-1;0;White, Mr. Percival Wayland;male;54;0;1;35281;77,2875;D26;S;;;Brunswick, ME
-1;0;White, Mr. Richard Frasar;male;21;0;1;35281;77,2875;D26;S;;169;Brunswick, ME
-1;1;White, Mrs. John Stuart (Ella Holmes);female;55;0;0;PC 17760;135,6333;C32;C;8;;New York, NY / Briarcliff Manor NY
-1;1;Wick, Miss. Mary Natalie;female;31;0;2;36928;164,8667;C7;S;8;;Youngstown, OH
-1;0;Wick, Mr. George Dennick;male;57;1;1;36928;164,8667;;S;;;Youngstown, OH
-1;1;Wick, Mrs. George Dennick (Mary Hitchcock);female;45;1;1;36928;164,8667;;S;8;;Youngstown, OH
-1;0;Widener, Mr. George Dunton;male;50;1;1;113503;211,5;C80;C;;;Elkins Park, PA
-1;0;Widener, Mr. Harry Elkins;male;27;0;2;113503;211,5;C82;C;;;Elkins Park, PA
-1;1;Widener, Mrs. George Dunton (Eleanor Elkins);female;50;1;1;113503;211,5;C80;C;4;;Elkins Park, PA
-1;1;Willard, Miss. Constance;female;21;0;0;113795;26,55;;S;8 10;;Duluth, MN
-1;0;Williams, Mr. Charles Duane;male;51;0;1;PC 17597;61,3792;;C;;;Geneva, Switzerland / Radnor, PA
-1;1;Williams, Mr. Richard Norris II;male;21;0;1;PC 17597;61,3792;;C;A;;Geneva, Switzerland / Radnor, PA
-1;0;Williams-Lambert, Mr. Fletcher Fellows;male;;0;0;113510;35;C128;S;;;London, England
-1;1;Wilson, Miss. Helen Alice;female;31;0;0;16966;134,5;E39 E41;C;3;;
-1;1;Woolner, Mr. Hugh;male;;0;0;19947;35,5;C52;S;D;;London, England
-1;0;Wright, Mr. George;male;62;0;0;113807;26,55;;S;;;Halifax, NS
-1;1;Young, Miss. Marie Grice;female;36;0;0;PC 17760;135,6333;C32;C;8;;New York, NY / Washington, DC
-2;0;Abelson, Mr. Samuel;male;30;1;0;P/PP 3381;24;;C;;;Russia New York, NY
-2;1;Abelson, Mrs. Samuel (Hannah Wizosky);female;28;1;0;P/PP 3381;24;;C;10;;Russia New York, NY
-2;0;Aldworth, Mr. Charles Augustus;male;30;0;0;248744;13;;S;;;Bryn Mawr, PA, USA
-2;0;Andrew, Mr. Edgardo Samuel;male;18;0;0;231945;11,5;;S;;;Buenos Aires, Argentina / New Jersey, NJ
-2;0;Andrew, Mr. Frank Thomas;male;25;0;0;C.A. 34050;10,5;;S;;;Cornwall, England Houghton, MI
-2;0;Angle, Mr. William A;male;34;1;0;226875;26;;S;;;Warwick, England
-2;1;"Angle, Mrs. William A (Florence ""Mary"" Agnes Hughes)";female;36;1;0;226875;26;;S;11;;Warwick, England
-2;0;Ashby, Mr. John;male;57;0;0;244346;13;;S;;;West Hoboken, NJ
-2;0;Bailey, Mr. Percy Andrew;male;18;0;0;29108;11,5;;S;;;Penzance, Cornwall / Akron, OH
-2;0;Baimbrigge, Mr. Charles Robert;male;23;0;0;C.A. 31030;10,5;;S;;;Guernsey
-2;1;Ball, Mrs. (Ada E Hall);female;36;0;0;28551;13;D;S;10;;Bristol, Avon / Jacksonville, FL
-2;0;Banfield, Mr. Frederick James;male;28;0;0;C.A./SOTON 34068;10,5;;S;;;Plymouth, Dorset / Houghton, MI
-2;0;Bateman, Rev. Robert James;male;51;0;0;S.O.P. 1166;12,525;;S;;174;Jacksonville, FL
-2;1;Beane, Mr. Edward;male;32;1;0;2908;26;;S;13;;Norwich / New York, NY
-2;1;Beane, Mrs. Edward (Ethel Clarke);female;19;1;0;2908;26;;S;13;;Norwich / New York, NY
-2;0;Beauchamp, Mr. Henry James;male;28;0;0;244358;26;;S;;;England
-2;1;Becker, Master. Richard F;male;1;2;1;230136;39;F4;S;11;;Guntur, India / Benton Harbour, MI
-2;1;Becker, Miss. Marion Louise;female;4;2;1;230136;39;F4;S;11;;Guntur, India / Benton Harbour, MI
-2;1;Becker, Miss. Ruth Elizabeth;female;12;2;1;230136;39;F4;S;13;;Guntur, India / Benton Harbour, MI
-2;1;Becker, Mrs. Allen Oliver (Nellie E Baumgardner);female;36;0;3;230136;39;F4;S;11;;Guntur, India / Benton Harbour, MI
-2;1;Beesley, Mr. Lawrence;male;34;0;0;248698;13;D56;S;13;;London
-2;1;Bentham, Miss. Lilian W;female;19;0;0;28404;13;;S;12;;Rochester, NY
-2;0;Berriman, Mr. William John;male;23;0;0;28425;13;;S;;;St Ives, Cornwall / Calumet, MI
-2;0;Botsford, Mr. William Hull;male;26;0;0;237670;13;;S;;;Elmira, NY / Orange, NJ
-2;0;Bowenur, Mr. Solomon;male;42;0;0;211535;13;;S;;;London
-2;0;Bracken, Mr. James H;male;27;0;0;220367;13;;S;;;Lake Arthur, Chavez County, NM
-2;1;"Brown, Miss. Amelia ""Mildred""";female;24;0;0;248733;13;F33;S;11;;London / Montreal, PQ
-2;1;Brown, Miss. Edith Eileen;female;15;0;2;29750;39;;S;14;;Cape Town, South Africa / Seattle, WA
-2;0;Brown, Mr. Thomas William Solomon;male;60;1;1;29750;39;;S;;;Cape Town, South Africa / Seattle, WA
-2;1;Brown, Mrs. Thomas William Solomon (Elizabeth Catherine Ford);female;40;1;1;29750;39;;S;14;;Cape Town, South Africa / Seattle, WA
-2;1;Bryhl, Miss. Dagmar Jenny Ingeborg ;female;20;1;0;236853;26;;S;12;;Skara, Sweden / Rockford, IL
-2;0;Bryhl, Mr. Kurt Arnold Gottfrid;male;25;1;0;236853;26;;S;;;Skara, Sweden / Rockford, IL
-2;1;Buss, Miss. Kate;female;36;0;0;27849;13;;S;9;;Sittingbourne, England / San Diego, CA
-2;0;Butler, Mr. Reginald Fenton;male;25;0;0;234686;13;;S;;97;Southsea, Hants
-2;0;Byles, Rev. Thomas Roussel Davids;male;42;0;0;244310;13;;S;;;London
-2;1;Bystrom, Mrs. (Karolina);female;42;0;0;236852;13;;S;;;New York, NY
-2;1;Caldwell, Master. Alden Gates;male;0,8333;0;2;248738;29;;S;13;;Bangkok, Thailand / Roseville, IL
-2;1;Caldwell, Mr. Albert Francis;male;26;1;1;248738;29;;S;13;;Bangkok, Thailand / Roseville, IL
-2;1;Caldwell, Mrs. Albert Francis (Sylvia Mae Harbaugh);female;22;1;1;248738;29;;S;13;;Bangkok, Thailand / Roseville, IL
-2;1;Cameron, Miss. Clear Annie;female;35;0;0;F.C.C. 13528;21;;S;14;;Mamaroneck, NY
-2;0;Campbell, Mr. William;male;;0;0;239853;0;;S;;;Belfast
-2;0;Carbines, Mr. William;male;19;0;0;28424;13;;S;;18;St Ives, Cornwall / Calumet, MI
-2;0;Carter, Mrs. Ernest Courtenay (Lilian Hughes);female;44;1;0;244252;26;;S;;;London
-2;0;Carter, Rev. Ernest Courtenay;male;54;1;0;244252;26;;S;;;London
-2;0;Chapman, Mr. Charles Henry;male;52;0;0;248731;13,5;;S;;130;Bronx, NY
-2;0;Chapman, Mr. John Henry;male;37;1;0;SC/AH 29037;26;;S;;17;Cornwall / Spokane, WA
-2;0;Chapman, Mrs. John Henry (Sara Elizabeth Lawry);female;29;1;0;SC/AH 29037;26;;S;;;Cornwall / Spokane, WA
-2;1;Christy, Miss. Julie Rachel;female;25;1;1;237789;30;;S;12;;London
-2;1;Christy, Mrs. (Alice Frances);female;45;0;2;237789;30;;S;12;;London
-2;0;Clarke, Mr. Charles Valentine;male;29;1;0;2003;26;;S;;;England / San Francisco, CA
-2;1;Clarke, Mrs. Charles V (Ada Maria Winfield);female;28;1;0;2003;26;;S;14;;England / San Francisco, CA
-2;0;Coleridge, Mr. Reginald Charles;male;29;0;0;W./C. 14263;10,5;;S;;;Hartford, Huntingdonshire
-2;0;Collander, Mr. Erik Gustaf;male;28;0;0;248740;13;;S;;;Helsinki, Finland Ashtabula, Ohio
-2;1;Collett, Mr. Sidney C Stuart;male;24;0;0;28034;10,5;;S;9;;London / Fort Byron, NY
-2;1;"Collyer, Miss. Marjorie ""Lottie""";female;8;0;2;C.A. 31921;26,25;;S;14;;Bishopstoke, Hants / Fayette Valley, ID
-2;0;Collyer, Mr. Harvey;male;31;1;1;C.A. 31921;26,25;;S;;;Bishopstoke, Hants / Fayette Valley, ID
-2;1;Collyer, Mrs. Harvey (Charlotte Annie Tate);female;31;1;1;C.A. 31921;26,25;;S;14;;Bishopstoke, Hants / Fayette Valley, ID
-2;1;Cook, Mrs. (Selena Rogers);female;22;0;0;W./C. 14266;10,5;F33;S;14;;Pennsylvania
-2;0;Corbett, Mrs. Walter H (Irene Colvin);female;30;0;0;237249;13;;S;;;Provo, UT
-2;0;Corey, Mrs. Percy C (Mary Phyllis Elizabeth Miller);female;;0;0;F.C.C. 13534;21;;S;;;Upper Burma, India Pittsburgh, PA
-2;0;"Cotterill, Mr. Henry ""Harry""";male;21;0;0;29107;11,5;;S;;;Penzance, Cornwall / Akron, OH
-2;0;Cunningham, Mr. Alfred Fleming;male;;0;0;239853;0;;S;;;Belfast
-2;1;Davies, Master. John Morgan Jr;male;8;1;1;C.A. 33112;36,75;;S;14;;St Ives, Cornwall / Hancock, MI
-2;0;Davies, Mr. Charles Henry;male;18;0;0;S.O.C. 14879;73,5;;S;;;Lyndhurst, England
-2;1;Davies, Mrs. John Morgan (Elizabeth Agnes Mary White) ;female;48;0;2;C.A. 33112;36,75;;S;14;;St Ives, Cornwall / Hancock, MI
-2;1;Davis, Miss. Mary;female;28;0;0;237668;13;;S;13;;London / Staten Island, NY
-2;0;de Brito, Mr. Jose Joaquim;male;32;0;0;244360;13;;S;;;Portugal / Sau Paulo, Brazil
-2;0;Deacon, Mr. Percy William;male;17;0;0;S.O.C. 14879;73,5;;S;;;
-2;0;del Carlo, Mr. Sebastiano;male;29;1;0;SC/PARIS 2167;27,7208;;C;;295;Lucca, Italy / California
-2;1;del Carlo, Mrs. Sebastiano (Argenia Genovesi);female;24;1;0;SC/PARIS 2167;27,7208;;C;12;;Lucca, Italy / California
-2;0;Denbury, Mr. Herbert;male;25;0;0;C.A. 31029;31,5;;S;;;Guernsey / Elizabeth, NJ
-2;0;Dibden, Mr. William;male;18;0;0;S.O.C. 14879;73,5;;S;;;New Forest, England
-2;1;Doling, Miss. Elsie;female;18;0;1;231919;23;;S;;;Southampton
-2;1;Doling, Mrs. John T (Ada Julia Bone);female;34;0;1;231919;23;;S;;;Southampton
-2;0;Downton, Mr. William James;male;54;0;0;28403;26;;S;;;Holley, NY
-2;1;Drew, Master. Marshall Brines;male;8;0;2;28220;32,5;;S;10;;Greenport, NY
-2;0;Drew, Mr. James Vivian;male;42;1;1;28220;32,5;;S;;;Greenport, NY
-2;1;Drew, Mrs. James Vivian (Lulu Thorne Christian);female;34;1;1;28220;32,5;;S;10;;Greenport, NY
-2;1;Duran y More, Miss. Asuncion;female;27;1;0;SC/PARIS 2149;13,8583;;C;12;;Barcelona, Spain / Havana, Cuba
-2;1;Duran y More, Miss. Florentina;female;30;1;0;SC/PARIS 2148;13,8583;;C;12;;Barcelona, Spain / Havana, Cuba
-2;0;Eitemiller, Mr. George Floyd;male;23;0;0;29751;13;;S;;;England / Detroit, MI
-2;0;Enander, Mr. Ingvar;male;21;0;0;236854;13;;S;;;Goteborg, Sweden / Rockford, IL
-2;0;Fahlstrom, Mr. Arne Jonas;male;18;0;0;236171;13;;S;;;Oslo, Norway Bayonne, NJ
-2;0;Faunthorpe, Mr. Harry;male;40;1;0;2926;26;;S;;286;England / Philadelphia, PA
-2;1;Faunthorpe, Mrs. Lizzie (Elizabeth Anne Wilkinson);female;29;1;0;2926;26;;S;16;;
-2;0;Fillbrook, Mr. Joseph Charles;male;18;0;0;C.A. 15185;10,5;;S;;;Cornwall / Houghton, MI
-2;0;Fox, Mr. Stanley Hubert;male;36;0;0;229236;13;;S;;236;Rochester, NY
-2;0;"Frost, Mr. Anthony Wood ""Archie""";male;;0;0;239854;0;;S;;;Belfast
-2;0;Funk, Miss. Annie Clemmer;female;38;0;0;237671;13;;S;;;Janjgir, India / Pennsylvania
-2;0;Fynney, Mr. Joseph J;male;35;0;0;239865;26;;S;;322;Liverpool / Montreal, PQ
-2;0;Gale, Mr. Harry;male;38;1;0;28664;21;;S;;;Cornwall / Clear Creek, CO
-2;0;Gale, Mr. Shadrach;male;34;1;0;28664;21;;S;;;Cornwall / Clear Creek, CO
-2;1;Garside, Miss. Ethel;female;34;0;0;243880;13;;S;12;;Brooklyn, NY
-2;0;Gaskell, Mr. Alfred;male;16;0;0;239865;26;;S;;;Liverpool / Montreal, PQ
-2;0;Gavey, Mr. Lawrence;male;26;0;0;31028;10,5;;S;;;Guernsey / Elizabeth, NJ
-2;0;Gilbert, Mr. William;male;47;0;0;C.A. 30769;10,5;;S;;;Cornwall
-2;0;Giles, Mr. Edgar;male;21;1;0;28133;11,5;;S;;;Cornwall / Camden, NJ
-2;0;Giles, Mr. Frederick Edward;male;21;1;0;28134;11,5;;S;;;Cornwall / Camden, NJ
-2;0;Giles, Mr. Ralph;male;24;0;0;248726;13,5;;S;;297;West Kensington, London
-2;0;Gill, Mr. John William;male;24;0;0;233866;13;;S;;155;Clevedon, England
-2;0;Gillespie, Mr. William Henry;male;34;0;0;12233;13;;S;;;Vancouver, BC
-2;0;Givard, Mr. Hans Kristensen;male;30;0;0;250646;13;;S;;305;
-2;0;Greenberg, Mr. Samuel;male;52;0;0;250647;13;;S;;19;Bronx, NY
-2;0;Hale, Mr. Reginald;male;30;0;0;250653;13;;S;;75;Auburn, NY
-2;1;Hamalainen, Master. Viljo;male;0,6667;1;1;250649;14,5;;S;4;;Detroit, MI
-2;1;Hamalainen, Mrs. William (Anna);female;24;0;2;250649;14,5;;S;4;;Detroit, MI
-2;0;Harbeck, Mr. William H;male;44;0;0;248746;13;;S;;35;Seattle, WA / Toledo, OH
-2;1;"Harper, Miss. Annie Jessie ""Nina""";female;6;0;1;248727;33;;S;11;;Denmark Hill, Surrey / Chicago
-2;0;Harper, Rev. John;male;28;0;1;248727;33;;S;;;Denmark Hill, Surrey / Chicago
-2;1;Harris, Mr. George;male;62;0;0;S.W./PP 752;10,5;;S;15;;London
-2;0;Harris, Mr. Walter;male;30;0;0;W/C 14208;10,5;;S;;;Walthamstow, England
-2;1;Hart, Miss. Eva Miriam;female;7;0;2;F.C.C. 13529;26,25;;S;14;;Ilford, Essex / Winnipeg, MB
-2;0;Hart, Mr. Benjamin;male;43;1;1;F.C.C. 13529;26,25;;S;;;Ilford, Essex / Winnipeg, MB
-2;1;Hart, Mrs. Benjamin (Esther Ada Bloomfield);female;45;1;1;F.C.C. 13529;26,25;;S;14;;Ilford, Essex / Winnipeg, MB
-2;1;Herman, Miss. Alice;female;24;1;2;220845;65;;S;9;;Somerset / Bernardsville, NJ
-2;1;Herman, Miss. Kate;female;24;1;2;220845;65;;S;9;;Somerset / Bernardsville, NJ
-2;0;Herman, Mr. Samuel;male;49;1;2;220845;65;;S;;;Somerset / Bernardsville, NJ
-2;1;Herman, Mrs. Samuel (Jane Laver);female;48;1;2;220845;65;;S;9;;Somerset / Bernardsville, NJ
-2;1;Hewlett, Mrs. (Mary D Kingcome) ;female;55;0;0;248706;16;;S;13;;India / Rapid City, SD
-2;0;Hickman, Mr. Leonard Mark;male;24;2;0;S.O.C. 14879;73,5;;S;;;West Hampstead, London / Neepawa, MB
-2;0;Hickman, Mr. Lewis;male;32;2;0;S.O.C. 14879;73,5;;S;;256;West Hampstead, London / Neepawa, MB
-2;0;Hickman, Mr. Stanley George;male;21;2;0;S.O.C. 14879;73,5;;S;;;West Hampstead, London / Neepawa, MB
-2;0;Hiltunen, Miss. Marta;female;18;1;1;250650;13;;S;;;Kontiolahti, Finland / Detroit, MI
-2;1;"Hocking, Miss. Ellen ""Nellie""";female;20;2;1;29105;23;;S;4;;Cornwall / Akron, OH
-2;0;Hocking, Mr. Richard George;male;23;2;1;29104;11,5;;S;;;Cornwall / Akron, OH
-2;0;Hocking, Mr. Samuel James Metcalfe;male;36;0;0;242963;13;;S;;;Devonport, England
-2;1;Hocking, Mrs. Elizabeth (Eliza Needs);female;54;1;3;29105;23;;S;4;;Cornwall / Akron, OH
-2;0;Hodges, Mr. Henry Price;male;50;0;0;250643;13;;S;;149;Southampton
-2;0;Hold, Mr. Stephen;male;44;1;0;26707;26;;S;;;England / Sacramento, CA
-2;1;Hold, Mrs. Stephen (Annie Margaret Hill);female;29;1;0;26707;26;;S;10;;England / Sacramento, CA
-2;0;Hood, Mr. Ambrose Jr;male;21;0;0;S.O.C. 14879;73,5;;S;;;New Forest, England
-2;1;Hosono, Mr. Masabumi;male;42;0;0;237798;13;;S;10;;Tokyo, Japan
-2;0;Howard, Mr. Benjamin;male;63;1;0;24065;26;;S;;;Swindon, England
-2;0;Howard, Mrs. Benjamin (Ellen Truelove Arman);female;60;1;0;24065;26;;S;;;Swindon, England
-2;0;Hunt, Mr. George Henry;male;33;0;0;SCO/W 1585;12,275;;S;;;Philadelphia, PA
-2;1;Ilett, Miss. Bertha;female;17;0;0;SO/C 14885;10,5;;S;;;Guernsey
-2;0;Jacobsohn, Mr. Sidney Samuel;male;42;1;0;243847;27;;S;;;London
-2;1;Jacobsohn, Mrs. Sidney Samuel (Amy Frances Christy);female;24;2;1;243847;27;;S;12;;London
-2;0;Jarvis, Mr. John Denzil;male;47;0;0;237565;15;;S;;;North Evington, England
-2;0;Jefferys, Mr. Clifford Thomas;male;24;2;0;C.A. 31029;31,5;;S;;;Guernsey / Elizabeth, NJ
-2;0;Jefferys, Mr. Ernest Wilfred;male;22;2;0;C.A. 31029;31,5;;S;;;Guernsey / Elizabeth, NJ
-2;0;Jenkin, Mr. Stephen Curnow;male;32;0;0;C.A. 33111;10,5;;S;;;St Ives, Cornwall / Houghton, MI
-2;1;Jerwan, Mrs. Amin S (Marie Marthe Thuillard);female;23;0;0;SC/AH Basle 541;13,7917;D;C;11;;New York, NY
-2;0;Kantor, Mr. Sinai;male;34;1;0;244367;26;;S;;283;Moscow / Bronx, NY
-2;1;Kantor, Mrs. Sinai (Miriam Sternin);female;24;1;0;244367;26;;S;12;;Moscow / Bronx, NY
-2;0;Karnes, Mrs. J Frank (Claire Bennett);female;22;0;0;F.C.C. 13534;21;;S;;;India / Pittsburgh, PA
-2;1;Keane, Miss. Nora A;female;;0;0;226593;12,35;E101;Q;10;;Harrisburg, PA
-2;0;Keane, Mr. Daniel;male;35;0;0;233734;12,35;;Q;;;
-2;1;"Kelly, Mrs. Florence ""Fannie""";female;45;0;0;223596;13,5;;S;9;;London / New York, NY
-2;0;Kirkland, Rev. Charles Leonard;male;57;0;0;219533;12,35;;Q;;;Glasgow / Bangor, ME
-2;0;Knight, Mr. Robert J;male;;0;0;239855;0;;S;;;Belfast
-2;0;Kvillner, Mr. Johan Henrik Johannesson;male;31;0;0;C.A. 18723;10,5;;S;;165;Sweden / Arlington, NJ
-2;0;Lahtinen, Mrs. William (Anna Sylfven);female;26;1;1;250651;26;;S;;;Minneapolis, MN
-2;0;Lahtinen, Rev. William;male;30;1;1;250651;26;;S;;;Minneapolis, MN
-2;0;Lamb, Mr. John Joseph;male;;0;0;240261;10,7083;;Q;;;
-2;1;Laroche, Miss. Louise;female;1;1;2;SC/Paris 2123;41,5792;;C;14;;Paris / Haiti
-2;1;Laroche, Miss. Simonne Marie Anne Andree;female;3;1;2;SC/Paris 2123;41,5792;;C;14;;Paris / Haiti
-2;0;Laroche, Mr. Joseph Philippe Lemercier;male;25;1;2;SC/Paris 2123;41,5792;;C;;;Paris / Haiti
-2;1;Laroche, Mrs. Joseph (Juliette Marie Louise Lafargue);female;22;1;2;SC/Paris 2123;41,5792;;C;14;;Paris / Haiti
-2;1;Lehmann, Miss. Bertha;female;17;0;0;SC 1748;12;;C;12;;Berne, Switzerland / Central City, IA
-2;1;Leitch, Miss. Jessie Wills;female;;0;0;248727;33;;S;11;;London / Chicago, IL
-2;1;Lemore, Mrs. (Amelia Milley);female;34;0;0;C.A. 34260;10,5;F33;S;14;;Chicago, IL
-2;0;Levy, Mr. Rene Jacques;male;36;0;0;SC/Paris 2163;12,875;D;C;;;Montreal, PQ
-2;0;Leyson, Mr. Robert William Norman;male;24;0;0;C.A. 29566;10,5;;S;;108;
-2;0;Lingane, Mr. John;male;61;0;0;235509;12,35;;Q;;;
-2;0;Louch, Mr. Charles Alexander;male;50;1;0;SC/AH 3085;26;;S;;121;Weston-Super-Mare, Somerset
-2;1;Louch, Mrs. Charles Alexander (Alice Adelaide Slow);female;42;1;0;SC/AH 3085;26;;S;;;Weston-Super-Mare, Somerset
-2;0;Mack, Mrs. (Mary);female;57;0;0;S.O./P.P. 3;10,5;E77;S;;52;Southampton / New York, NY
-2;0;Malachard, Mr. Noel;male;;0;0;237735;15,0458;D;C;;;Paris
-2;1;Mallet, Master. Andre;male;1;0;2;S.C./PARIS 2079;37,0042;;C;10;;Paris / Montreal, PQ
-2;0;Mallet, Mr. Albert;male;31;1;1;S.C./PARIS 2079;37,0042;;C;;;Paris / Montreal, PQ
-2;1;Mallet, Mrs. Albert (Antoinette Magnin);female;24;1;1;S.C./PARIS 2079;37,0042;;C;10;;Paris / Montreal, PQ
-2;0;Mangiavacchi, Mr. Serafino Emilio;male;;0;0;SC/A.3 2861;15,5792;;C;;;New York, NY
-2;0;Matthews, Mr. William John;male;30;0;0;28228;13;;S;;;St Austall, Cornwall
-2;0;Maybery, Mr. Frank Hubert;male;40;0;0;239059;16;;S;;;Weston-Super-Mare / Moose Jaw, SK
-2;0;McCrae, Mr. Arthur Gordon;male;32;0;0;237216;13,5;;S;;209;Sydney, Australia
-2;0;McCrie, Mr. James Matthew;male;30;0;0;233478;13;;S;;;Sarnia, ON
-2;0;McKane, Mr. Peter David;male;46;0;0;28403;26;;S;;;Rochester, NY
-2;1;Mellinger, Miss. Madeleine Violet;female;13;0;1;250644;19,5;;S;14;;England / Bennington, VT
-2;1;Mellinger, Mrs. (Elizabeth Anne Maidment);female;41;0;1;250644;19,5;;S;14;;England / Bennington, VT
-2;1;Mellors, Mr. William John;male;19;0;0;SW/PP 751;10,5;;S;B;;Chelsea, London
-2;0;Meyer, Mr. August;male;39;0;0;248723;13;;S;;;Harrow-on-the-Hill, Middlesex
-2;0;Milling, Mr. Jacob Christian;male;48;0;0;234360;13;;S;;271;Copenhagen, Denmark
-2;0;Mitchell, Mr. Henry Michael;male;70;0;0;C.A. 24580;10,5;;S;;;Guernsey / Montclair, NJ and/or Toledo, Ohio
-2;0;Montvila, Rev. Juozas;male;27;0;0;211536;13;;S;;;Worcester, MA
-2;0;Moraweck, Dr. Ernest;male;54;0;0;29011;14;;S;;;Frankfort, KY
-2;0;"Morley, Mr. Henry Samuel (""Mr Henry Marshall"")";male;39;0;0;250655;26;;S;;;
-2;0;Mudd, Mr. Thomas Charles;male;16;0;0;S.O./P.P. 3;10,5;;S;;;Halesworth, England
-2;0;Myles, Mr. Thomas Francis;male;62;0;0;240276;9,6875;;Q;;;Cambridge, MA
-2;0;Nasser, Mr. Nicholas;male;32,5;1;0;237736;30,0708;;C;;43;New York, NY
-2;1;Nasser, Mrs. Nicholas (Adele Achem);female;14;1;0;237736;30,0708;;C;;;New York, NY
-2;1;Navratil, Master. Edmond Roger;male;2;1;1;230080;26;F2;S;D;;Nice, France
-2;1;Navratil, Master. Michel M;male;3;1;1;230080;26;F2;S;D;;Nice, France
-2;0;"Navratil, Mr. Michel (""Louis M Hoffman"")";male;36,5;0;2;230080;26;F2;S;;15;Nice, France
-2;0;Nesson, Mr. Israel;male;26;0;0;244368;13;F2;S;;;Boston, MA
-2;0;Nicholls, Mr. Joseph Charles;male;19;1;1;C.A. 33112;36,75;;S;;101;Cornwall / Hancock, MI
-2;0;Norman, Mr. Robert Douglas;male;28;0;0;218629;13,5;;S;;287;Glasgow
-2;1;"Nourney, Mr. Alfred (""Baron von Drachstedt"")";male;20;0;0;SC/PARIS 2166;13,8625;D38;C;7;;Cologne, Germany
-2;1;Nye, Mrs. (Elizabeth Ramell);female;29;0;0;C.A. 29395;10,5;F33;S;11;;Folkstone, Kent / New York, NY
-2;0;Otter, Mr. Richard;male;39;0;0;28213;13;;S;;;Middleburg Heights, OH
-2;1;Oxenham, Mr. Percy Thomas;male;22;0;0;W./C. 14260;10,5;;S;13;;Pondersend, England / New Durham, NJ
-2;1;Padro y Manent, Mr. Julian;male;;0;0;SC/PARIS 2146;13,8625;;C;9;;Spain / Havana, Cuba
-2;0;Pain, Dr. Alfred;male;23;0;0;244278;10,5;;S;;;Hamilton, ON
-2;1;Pallas y Castello, Mr. Emilio;male;29;0;0;SC/PARIS 2147;13,8583;;C;9;;Spain / Havana, Cuba
-2;0;Parker, Mr. Clifford Richard;male;28;0;0;SC 14888;10,5;;S;;;St Andrews, Guernsey
-2;0;"Parkes, Mr. Francis ""Frank""";male;;0;0;239853;0;;S;;;Belfast
-2;1;Parrish, Mrs. (Lutie Davis);female;50;0;1;230433;26;;S;12;;Woodford County, KY
-2;0;Pengelly, Mr. Frederick William;male;19;0;0;28665;10,5;;S;;;Gunnislake, England / Butte, MT
-2;0;Pernot, Mr. Rene;male;;0;0;SC/PARIS 2131;15,05;;C;;;
-2;0;Peruschitz, Rev. Joseph Maria;male;41;0;0;237393;13;;S;;;
-2;1;Phillips, Miss. Alice Frances Louisa;female;21;0;1;S.O./P.P. 2;21;;S;12;;Ilfracombe, Devon
-2;1;"Phillips, Miss. Kate Florence (""Mrs Kate Louise Phillips Marshall"")";female;19;0;0;250655;26;;S;11;;Worcester, England
-2;0;Phillips, Mr. Escott Robert;male;43;0;1;S.O./P.P. 2;21;;S;;;Ilfracombe, Devon
-2;1;Pinsky, Mrs. (Rosa);female;32;0;0;234604;13;;S;9;;Russia
-2;0;Ponesell, Mr. Martin;male;34;0;0;250647;13;;S;;;Denmark / New York, NY
-2;1;Portaluppi, Mr. Emilio Ilario Giuseppe;male;30;0;0;C.A. 34644;12,7375;;C;14;;Milford, NH
-2;0;Pulbaum, Mr. Franz;male;27;0;0;SC/PARIS 2168;15,0333;;C;;;Paris
-2;1;Quick, Miss. Phyllis May;female;2;1;1;26360;26;;S;11;;Plymouth, Devon / Detroit, MI
-2;1;Quick, Miss. Winifred Vera;female;8;1;1;26360;26;;S;11;;Plymouth, Devon / Detroit, MI
-2;1;Quick, Mrs. Frederick Charles (Jane Richards);female;33;0;2;26360;26;;S;11;;Plymouth, Devon / Detroit, MI
-2;0;Reeves, Mr. David;male;36;0;0;C.A. 17248;10,5;;S;;;Brighton, Sussex
-2;0;Renouf, Mr. Peter Henry;male;34;1;0;31027;21;;S;12;;Elizabeth, NJ
-2;1;Renouf, Mrs. Peter Henry (Lillian Jefferys);female;30;3;0;31027;21;;S;;;Elizabeth, NJ
-2;1;Reynaldo, Ms. Encarnacion;female;28;0;0;230434;13;;S;9;;Spain
-2;0;Richard, Mr. Emile;male;23;0;0;SC/PARIS 2133;15,0458;;C;;;Paris / Montreal, PQ
-2;1;Richards, Master. George Sibley;male;0,8333;1;1;29106;18,75;;S;4;;Cornwall / Akron, OH
-2;1;Richards, Master. William Rowe;male;3;1;1;29106;18,75;;S;4;;Cornwall / Akron, OH
-2;1;Richards, Mrs. Sidney (Emily Hocking);female;24;2;3;29106;18,75;;S;4;;Cornwall / Akron, OH
-2;1;Ridsdale, Miss. Lucy;female;50;0;0;W./C. 14258;10,5;;S;13;;London, England / Marietta, Ohio and Milwaukee, WI
-2;0;Rogers, Mr. Reginald Harry;male;19;0;0;28004;10,5;;S;;;
-2;1;Rugg, Miss. Emily;female;21;0;0;C.A. 31026;10,5;;S;12;;Guernsey / Wilmington, DE
-2;0;Schmidt, Mr. August;male;26;0;0;248659;13;;S;;;Newark, NJ
-2;0;Sedgwick, Mr. Charles Frederick Waddington;male;25;0;0;244361;13;;S;;;Liverpool
-2;0;Sharp, Mr. Percival James R;male;27;0;0;244358;26;;S;;;Hornsey, England
-2;1;Shelley, Mrs. William (Imanita Parrish Hall);female;25;0;1;230433;26;;S;12;;Deer Lodge, MT
-2;1;Silven, Miss. Lyyli Karoliina;female;18;0;2;250652;13;;S;16;;Finland / Minneapolis, MN
-2;1;Sincock, Miss. Maude;female;20;0;0;C.A. 33112;36,75;;S;11;;Cornwall / Hancock, MI
-2;1;Sinkkonen, Miss. Anna;female;30;0;0;250648;13;;S;10;;Finland / Washington, DC
-2;0;Sjostedt, Mr. Ernst Adolf;male;59;0;0;237442;13,5;;S;;;Sault St Marie, ON
-2;1;Slayter, Miss. Hilda Mary;female;30;0;0;234818;12,35;;Q;13;;Halifax, NS
-2;0;Slemen, Mr. Richard James;male;35;0;0;28206;10,5;;S;;;Cornwall
-2;1;Smith, Miss. Marion Elsie;female;40;0;0;31418;13;;S;9;;
-2;0;Sobey, Mr. Samuel James Hayden;male;25;0;0;C.A. 29178;13;;S;;;Cornwall / Houghton, MI
-2;0;Stanton, Mr. Samuel Ward;male;41;0;0;237734;15,0458;;C;;;New York, NY
-2;0;Stokes, Mr. Philip Joseph;male;25;0;0;F.C.C. 13540;10,5;;S;;81;Catford, Kent / Detroit, MI
-2;0;Swane, Mr. George;male;18,5;0;0;248734;13;F;S;;294;
-2;0;Sweet, Mr. George Frederick;male;14;0;0;220845;65;;S;;;Somerset / Bernardsville, NJ
-2;1;Toomey, Miss. Ellen;female;50;0;0;F.C.C. 13531;10,5;;S;9;;Indianapolis, IN
-2;0;Troupiansky, Mr. Moses Aaron;male;23;0;0;233639;13;;S;;;
-2;1;Trout, Mrs. William H (Jessie L);female;28;0;0;240929;12,65;;S;;;Columbus, OH
-2;1;"Troutt, Miss. Edwina Celia ""Winnie""";female;27;0;0;34218;10,5;E101;S;16;;Bath, England / Massachusetts
-2;0;Turpin, Mr. William John Robert;male;29;1;0;11668;21;;S;;;Plymouth, England
-2;0;Turpin, Mrs. William John Robert (Dorothy Ann Wonnacott);female;27;1;0;11668;21;;S;;;Plymouth, England
-2;0;Veal, Mr. James;male;40;0;0;28221;13;;S;;;Barre, Co Washington, VT
-2;1;Walcroft, Miss. Nellie;female;31;0;0;F.C.C. 13528;21;;S;14;;Mamaroneck, NY
-2;0;Ware, Mr. John James;male;30;1;0;CA 31352;21;;S;;;Bristol, England / New Britain, CT
-2;0;Ware, Mr. William Jeffery;male;23;1;0;28666;10,5;;S;;;
-2;1;Ware, Mrs. John James (Florence Louise Long);female;31;0;0;CA 31352;21;;S;10;;Bristol, England / New Britain, CT
-2;0;Watson, Mr. Ennis Hastings;male;;0;0;239856;0;;S;;;Belfast
-2;1;Watt, Miss. Bertha J;female;12;0;0;C.A. 33595;15,75;;S;9;;Aberdeen / Portland, OR
-2;1;"Watt, Mrs. James (Elizabeth ""Bessie"" Inglis Milne)";female;40;0;0;C.A. 33595;15,75;;S;9;;Aberdeen / Portland, OR
-2;1;Webber, Miss. Susan;female;32,5;0;0;27267;13;E101;S;12;;England / Hartford, CT
-2;0;Weisz, Mr. Leopold;male;27;1;0;228414;26;;S;;293;Bromsgrove, England / Montreal, PQ
-2;1;Weisz, Mrs. Leopold (Mathilde Francoise Pede);female;29;1;0;228414;26;;S;10;;Bromsgrove, England / Montreal, PQ
-2;1;Wells, Master. Ralph Lester;male;2;1;1;29103;23;;S;14;;Cornwall / Akron, OH
-2;1;Wells, Miss. Joan;female;4;1;1;29103;23;;S;14;;Cornwall / Akron, OH
-2;1;"Wells, Mrs. Arthur Henry (""Addie"" Dart Trevaskis)";female;29;0;2;29103;23;;S;14;;Cornwall / Akron, OH
-2;1;West, Miss. Barbara J;female;0,9167;1;2;C.A. 34651;27,75;;S;10;;Bournmouth, England
-2;1;West, Miss. Constance Mirium;female;5;1;2;C.A. 34651;27,75;;S;10;;Bournmouth, England
-2;0;West, Mr. Edwy Arthur;male;36;1;2;C.A. 34651;27,75;;S;;;Bournmouth, England
-2;1;West, Mrs. Edwy Arthur (Ada Mary Worth);female;33;1;2;C.A. 34651;27,75;;S;10;;Bournmouth, England
-2;0;Wheadon, Mr. Edward H;male;66;0;0;C.A. 24579;10,5;;S;;;Guernsey, England / Edgewood, RI
-2;0;"Wheeler, Mr. Edwin ""Frederick""";male;;0;0;SC/PARIS 2159;12,875;;S;;;
-2;1;Wilhelms, Mr. Charles;male;31;0;0;244270;13;;S;9;;London, England
-2;1;Williams, Mr. Charles Eugene;male;;0;0;244373;13;;S;14;;Harrow, England
-2;1;Wright, Miss. Marion;female;26;0;0;220844;13,5;;S;9;;Yoevil, England / Cottage Grove, OR
-2;0;"Yrois, Miss. Henriette (""Mrs Harbeck"")";female;24;0;0;248747;13;;S;;;Paris
-3;0;Abbing, Mr. Anthony;male;42;0;0;C.A. 5547;7,55;;S;;;
-3;0;Abbott, Master. Eugene Joseph;male;13;0;2;C.A. 2673;20,25;;S;;;East Providence, RI
-3;0;Abbott, Mr. Rossmore Edward;male;16;1;1;C.A. 2673;20,25;;S;;190;East Providence, RI
-3;1;Abbott, Mrs. Stanton (Rosa Hunt);female;35;1;1;C.A. 2673;20,25;;S;A;;East Providence, RI
-3;1;Abelseth, Miss. Karen Marie;female;16;0;0;348125;7,65;;S;16;;Norway Los Angeles, CA
-3;1;Abelseth, Mr. Olaus Jorgensen;male;25;0;0;348122;7,65;F G63;S;A;;Perkins County, SD
-3;1;Abrahamsson, Mr. Abraham August Johannes;male;20;0;0;SOTON/O2 3101284;7,925;;S;15;;Taalintehdas, Finland Hoboken, NJ
-3;1;Abrahim, Mrs. Joseph (Sophie Halaut Easu);female;18;0;0;2657;7,2292;;C;C;;Greensburg, PA
-3;0;Adahl, Mr. Mauritz Nils Martin;male;30;0;0;C 7076;7,25;;S;;72;Asarum, Sweden Brooklyn, NY
-3;0;Adams, Mr. John;male;26;0;0;341826;8,05;;S;;103;Bournemouth, England
-3;0;Ahlin, Mrs. Johan (Johanna Persdotter Larsson);female;40;1;0;7546;9,475;;S;;;Sweden Akeley, MN
-3;1;Aks, Master. Philip Frank;male;0,8333;0;1;392091;9,35;;S;11;;London, England Norfolk, VA
-3;1;Aks, Mrs. Sam (Leah Rosen);female;18;0;1;392091;9,35;;S;13;;London, England Norfolk, VA
-3;1;Albimona, Mr. Nassef Cassem;male;26;0;0;2699;18,7875;;C;15;;Syria Fredericksburg, VA
-3;0;Alexander, Mr. William;male;26;0;0;3474;7,8875;;S;;;England Albion, NY
-3;0;Alhomaki, Mr. Ilmari Rudolf;male;20;0;0;SOTON/O2 3101287;7,925;;S;;;Salo, Finland Astoria, OR
-3;0;Ali, Mr. Ahmed;male;24;0;0;SOTON/O.Q. 3101311;7,05;;S;;;
-3;0;Ali, Mr. William;male;25;0;0;SOTON/O.Q. 3101312;7,05;;S;;79;Argentina
-3;0;Allen, Mr. William Henry;male;35;0;0;373450;8,05;;S;;;Lower Clapton, Middlesex or Erdington, Birmingham
-3;0;Allum, Mr. Owen George;male;18;0;0;2223;8,3;;S;;259;Windsor, England New York, NY
-3;0;Andersen, Mr. Albert Karvin;male;32;0;0;C 4001;22,525;;S;;260;Bergen, Norway
-3;1;Andersen-Jensen, Miss. Carla Christine Nielsine;female;19;1;0;350046;7,8542;;S;16;;
-3;0;Andersson, Master. Sigvard Harald Elias;male;4;4;2;347082;31,275;;S;;;Sweden Winnipeg, MN
-3;0;Andersson, Miss. Ebba Iris Alfrida;female;6;4;2;347082;31,275;;S;;;Sweden Winnipeg, MN
-3;0;Andersson, Miss. Ellis Anna Maria;female;2;4;2;347082;31,275;;S;;;Sweden Winnipeg, MN
-3;1;Andersson, Miss. Erna Alexandra;female;17;4;2;3101281;7,925;;S;D;;Ruotsinphyhtaa, Finland New York, NY
-3;0;Andersson, Miss. Ida Augusta Margareta;female;38;4;2;347091;7,775;;S;;;Vadsbro, Sweden Ministee, MI
-3;0;Andersson, Miss. Ingeborg Constanzia;female;9;4;2;347082;31,275;;S;;;Sweden Winnipeg, MN
-3;0;Andersson, Miss. Sigrid Elisabeth;female;11;4;2;347082;31,275;;S;;;Sweden Winnipeg, MN
-3;0;Andersson, Mr. Anders Johan;male;39;1;5;347082;31,275;;S;;;Sweden Winnipeg, MN
-3;1;"Andersson, Mr. August Edvard (""Wennerstrom"")";male;27;0;0;350043;7,7958;;S;A;;
-3;0;Andersson, Mr. Johan Samuel;male;26;0;0;347075;7,775;;S;;;Hartford, CT
-3;0;Andersson, Mrs. Anders Johan (Alfrida Konstantia Brogren);female;39;1;5;347082;31,275;;S;;;Sweden Winnipeg, MN
-3;0;Andreasson, Mr. Paul Edvin;male;20;0;0;347466;7,8542;;S;;;Sweden Chicago, IL
-3;0;Angheloff, Mr. Minko;male;26;0;0;349202;7,8958;;S;;;Bulgaria Chicago, IL
-3;0;Arnold-Franchi, Mr. Josef;male;25;1;0;349237;17,8;;S;;;Altdorf, Switzerland
-3;0;Arnold-Franchi, Mrs. Josef (Josefine Franchi);female;18;1;0;349237;17,8;;S;;;Altdorf, Switzerland
-3;0;Aronsson, Mr. Ernst Axel Algot;male;24;0;0;349911;7,775;;S;;;Sweden Joliet, IL
-3;0;Asim, Mr. Adola;male;35;0;0;SOTON/O.Q. 3101310;7,05;;S;;;
-3;0;Asplund, Master. Carl Edgar;male;5;4;2;347077;31,3875;;S;;;Sweden  Worcester, MA
-3;0;Asplund, Master. Clarence Gustaf Hugo;male;9;4;2;347077;31,3875;;S;;;Sweden Worcester, MA
-3;1;Asplund, Master. Edvin Rojj Felix;male;3;4;2;347077;31,3875;;S;15;;Sweden Worcester, MA
-3;0;Asplund, Master. Filip Oscar;male;13;4;2;347077;31,3875;;S;;;Sweden Worcester, MA
-3;1;Asplund, Miss. Lillian Gertrud;female;5;4;2;347077;31,3875;;S;15;;Sweden Worcester, MA
-3;0;Asplund, Mr. Carl Oscar Vilhelm Gustafsson;male;40;1;5;347077;31,3875;;S;;142;Sweden  Worcester, MA
-3;1;Asplund, Mr. Johan Charles;male;23;0;0;350054;7,7958;;S;13;;Oskarshamn, Sweden Minneapolis, MN
-3;1;Asplund, Mrs. Carl Oscar (Selma Augusta Emilia Johansson);female;38;1;5;347077;31,3875;;S;15;;Sweden  Worcester, MA
-3;1;"Assaf Khalil, Mrs. Mariana (""Miriam"")";female;45;0;0;2696;7,225;;C;C;;Ottawa, ON
-3;0;Assaf, Mr. Gerios;male;21;0;0;2692;7,225;;C;;;Ottawa, ON
-3;0;Assam, Mr. Ali;male;23;0;0;SOTON/O.Q. 3101309;7,05;;S;;;
-3;0;Attalah, Miss. Malake;female;17;0;0;2627;14,4583;;C;;;
-3;0;Attalah, Mr. Sleiman;male;30;0;0;2694;7,225;;C;;;Ottawa, ON
-3;0;Augustsson, Mr. Albert;male;23;0;0;347468;7,8542;;S;;;Krakoryd, Sweden Bloomington, IL
-3;1;Ayoub, Miss. Banoura;female;13;0;0;2687;7,2292;;C;C;;Syria Youngstown, OH
-3;0;Baccos, Mr. Raffull;male;20;0;0;2679;7,225;;C;;;
-3;0;Backstrom, Mr. Karl Alfred;male;32;1;0;3101278;15,85;;S;D;;Ruotsinphytaa, Finland New York, NY
-3;1;Backstrom, Mrs. Karl Alfred (Maria Mathilda Gustafsson);female;33;3;0;3101278;15,85;;S;;;Ruotsinphytaa, Finland New York, NY
-3;1;Baclini, Miss. Eugenie;female;0,75;2;1;2666;19,2583;;C;C;;Syria New York, NY
-3;1;Baclini, Miss. Helene Barbara;female;0,75;2;1;2666;19,2583;;C;C;;Syria New York, NY
-3;1;Baclini, Miss. Marie Catherine;female;5;2;1;2666;19,2583;;C;C;;Syria New York, NY
-3;1;Baclini, Mrs. Solomon (Latifa Qurban);female;24;0;3;2666;19,2583;;C;C;;Syria New York, NY
-3;1;Badman, Miss. Emily Louisa;female;18;0;0;A/4 31416;8,05;;S;C;;London Skanteales, NY
-3;0;Badt, Mr. Mohamed;male;40;0;0;2623;7,225;;C;;;
-3;0;Balkic, Mr. Cerin;male;26;0;0;349248;7,8958;;S;;;
-3;1;Barah, Mr. Hanna Assi;male;20;0;0;2663;7,2292;;C;15;;
-3;0;Barbara, Miss. Saiide;female;18;0;1;2691;14,4542;;C;;;Syria Ottawa, ON
-3;0;Barbara, Mrs. (Catherine David);female;45;0;1;2691;14,4542;;C;;;Syria Ottawa, ON
-3;0;Barry, Miss. Julia;female;27;0;0;330844;7,8792;;Q;;;New York, NY
-3;0;Barton, Mr. David John;male;22;0;0;324669;8,05;;S;;;England New York, NY
-3;0;Beavan, Mr. William Thomas;male;19;0;0;323951;8,05;;S;;;England
-3;0;Bengtsson, Mr. John Viktor;male;26;0;0;347068;7,775;;S;;;Krakudden, Sweden Moune, IL
-3;0;Berglund, Mr. Karl Ivar Sven;male;22;0;0;PP 4348;9,35;;S;;;Tranvik, Finland New York
-3;0;Betros, Master. Seman;male;;0;0;2622;7,2292;;C;;;
-3;0;Betros, Mr. Tannous;male;20;0;0;2648;4,0125;;C;;;Syria
-3;1;Bing, Mr. Lee;male;32;0;0;1601;56,4958;;S;C;;Hong Kong New York, NY
-3;0;Birkeland, Mr. Hans Martin Monsen;male;21;0;0;312992;7,775;;S;;;Brennes, Norway New York
-3;0;Bjorklund, Mr. Ernst Herbert;male;18;0;0;347090;7,75;;S;;;Stockholm, Sweden New York
-3;0;Bostandyeff, Mr. Guentcho;male;26;0;0;349224;7,8958;;S;;;Bulgaria Chicago, IL
-3;0;Boulos, Master. Akar;male;6;1;1;2678;15,2458;;C;;;Syria Kent, ON
-3;0;Boulos, Miss. Nourelain;female;9;1;1;2678;15,2458;;C;;;Syria Kent, ON
-3;0;Boulos, Mr. Hanna;male;;0;0;2664;7,225;;C;;;Syria
-3;0;Boulos, Mrs. Joseph (Sultana);female;;0;2;2678;15,2458;;C;;;Syria Kent, ON
-3;0;Bourke, Miss. Mary;female;;0;2;364848;7,75;;Q;;;Ireland Chicago, IL
-3;0;Bourke, Mr. John;male;40;1;1;364849;15,5;;Q;;;Ireland Chicago, IL
-3;0;Bourke, Mrs. John (Catherine);female;32;1;1;364849;15,5;;Q;;;Ireland Chicago, IL
-3;0;"Bowen, Mr. David John ""Dai""";male;21;0;0;54636;16,1;;S;;;Treherbert, Cardiff, Wales
-3;1;Bradley, Miss. Bridget Delia;female;22;0;0;334914;7,725;;Q;13;;Kingwilliamstown, Co Cork, Ireland Glens Falls, NY
-3;0;Braf, Miss. Elin Ester Maria;female;20;0;0;347471;7,8542;;S;;;Medeltorp, Sweden Chicago, IL
-3;0;Braund, Mr. Lewis Richard;male;29;1;0;3460;7,0458;;S;;;Bridgerule, Devon
-3;0;Braund, Mr. Owen Harris;male;22;1;0;A/5 21171;7,25;;S;;;Bridgerule, Devon
-3;0;Brobeck, Mr. Karl Rudolf;male;22;0;0;350045;7,7958;;S;;;Sweden Worcester, MA
-3;0;Brocklebank, Mr. William Alfred;male;35;0;0;364512;8,05;;S;;;Broomfield, Chelmsford, England
-3;0;Buckley, Miss. Katherine;female;18,5;0;0;329944;7,2833;;Q;;299;Co Cork, Ireland Roxbury, MA
-3;1;Buckley, Mr. Daniel;male;21;0;0;330920;7,8208;;Q;13;;Kingwilliamstown, Co Cork, Ireland New York, NY
-3;0;Burke, Mr. Jeremiah;male;19;0;0;365222;6,75;;Q;;;Co Cork, Ireland Charlestown, MA
-3;0;Burns, Miss. Mary Delia;female;18;0;0;330963;7,8792;;Q;;;Co Sligo, Ireland New York, NY
-3;0;Cacic, Miss. Manda;female;21;0;0;315087;8,6625;;S;;;
-3;0;Cacic, Miss. Marija;female;30;0;0;315084;8,6625;;S;;;
-3;0;Cacic, Mr. Jego Grga;male;18;0;0;315091;8,6625;;S;;;
-3;0;Cacic, Mr. Luka;male;38;0;0;315089;8,6625;;S;;;Croatia
-3;0;Calic, Mr. Jovo;male;17;0;0;315093;8,6625;;S;;;
-3;0;Calic, Mr. Petar;male;17;0;0;315086;8,6625;;S;;;
-3;0;Canavan, Miss. Mary;female;21;0;0;364846;7,75;;Q;;;
-3;0;Canavan, Mr. Patrick;male;21;0;0;364858;7,75;;Q;;;Ireland Philadelphia, PA
-3;0;Cann, Mr. Ernest Charles;male;21;0;0;A./5. 2152;8,05;;S;;;
-3;0;Caram, Mr. Joseph;male;;1;0;2689;14,4583;;C;;;Ottawa, ON
-3;0;Caram, Mrs. Joseph (Maria Elias);female;;1;0;2689;14,4583;;C;;;Ottawa, ON
-3;0;Carlsson, Mr. August Sigfrid;male;28;0;0;350042;7,7958;;S;;;Dagsas, Sweden Fower, MN
-3;0;Carlsson, Mr. Carl Robert;male;24;0;0;350409;7,8542;;S;;;Goteborg, Sweden Huntley, IL
-3;1;"Carr, Miss. Helen ""Ellen""";female;16;0;0;367231;7,75;;Q;16;;Co Longford, Ireland New York, NY
-3;0;Carr, Miss. Jeannie;female;37;0;0;368364;7,75;;Q;;;Co Sligo, Ireland Hartford, CT
-3;0;Carver, Mr. Alfred John;male;28;0;0;392095;7,25;;S;;;St Denys, Southampton, Hants
-3;0;Celotti, Mr. Francesco;male;24;0;0;343275;8,05;;S;;;London
-3;0;Charters, Mr. David;male;21;0;0;A/5. 13032;7,7333;;Q;;;Ireland New York, NY
-3;1;Chip, Mr. Chang;male;32;0;0;1601;56,4958;;S;C;;Hong Kong New York, NY
-3;0;Christmann, Mr. Emil;male;29;0;0;343276;8,05;;S;;;
-3;0;Chronopoulos, Mr. Apostolos;male;26;1;0;2680;14,4542;;C;;;Greece
-3;0;Chronopoulos, Mr. Demetrios;male;18;1;0;2680;14,4542;;C;;;Greece
-3;0;Coelho, Mr. Domingos Fernandeo;male;20;0;0;SOTON/O.Q. 3101307;7,05;;S;;;Portugal
-3;1;"Cohen, Mr. Gurshon ""Gus""";male;18;0;0;A/5 3540;8,05;;S;12;;London Brooklyn, NY
-3;0;Colbert, Mr. Patrick;male;24;0;0;371109;7,25;;Q;;;Co Limerick, Ireland Sherbrooke, PQ
-3;0;Coleff, Mr. Peju;male;36;0;0;349210;7,4958;;S;;;Bulgaria Chicago, IL
-3;0;Coleff, Mr. Satio;male;24;0;0;349209;7,4958;;S;;;
-3;0;Conlon, Mr. Thomas Henry;male;31;0;0;21332;7,7333;;Q;;;Philadelphia, PA
-3;0;Connaghton, Mr. Michael;male;31;0;0;335097;7,75;;Q;;;Ireland Brooklyn, NY
-3;1;Connolly, Miss. Kate;female;22;0;0;370373;7,75;;Q;13;;Ireland
-3;0;Connolly, Miss. Kate;female;30;0;0;330972;7,6292;;Q;;;Ireland
-3;0;Connors, Mr. Patrick;male;70,5;0;0;370369;7,75;;Q;;171;
-3;0;Cook, Mr. Jacob;male;43;0;0;A/5 3536;8,05;;S;;;
-3;0;Cor, Mr. Bartol;male;35;0;0;349230;7,8958;;S;;;Austria
-3;0;Cor, Mr. Ivan;male;27;0;0;349229;7,8958;;S;;;Austria
-3;0;Cor, Mr. Liudevit;male;19;0;0;349231;7,8958;;S;;;Austria
-3;0;Corn, Mr. Harry;male;30;0;0;SOTON/OQ 392090;8,05;;S;;;London
-3;1;"Coutts, Master. Eden Leslie ""Neville""";male;9;1;1;C.A. 37671;15,9;;S;2;;England Brooklyn, NY
-3;1;"Coutts, Master. William Loch ""William""";male;3;1;1;C.A. 37671;15,9;;S;2;;England Brooklyn, NY
-3;1;"Coutts, Mrs. William (Winnie ""Minnie"" Treanor)";female;36;0;2;C.A. 37671;15,9;;S;2;;England Brooklyn, NY
-3;0;Coxon, Mr. Daniel;male;59;0;0;364500;7,25;;S;;;Merrill, WI
-3;0;Crease, Mr. Ernest James;male;19;0;0;S.P. 3464;8,1583;;S;;;Bristol, England Cleveland, OH
-3;1;Cribb, Miss. Laura Alice;female;17;0;1;371362;16,1;;S;12;;Bournemouth, England Newark, NJ
-3;0;Cribb, Mr. John Hatfield;male;44;0;1;371362;16,1;;S;;;Bournemouth, England Newark, NJ
-3;0;Culumovic, Mr. Jeso;male;17;0;0;315090;8,6625;;S;;;Austria-Hungary
-3;0;Daher, Mr. Shedid;male;22,5;0;0;2698;7,225;;C;;9;
-3;1;Dahl, Mr. Karl Edwart;male;45;0;0;7598;8,05;;S;15;;Australia Fingal, ND
-3;0;Dahlberg, Miss. Gerda Ulrika;female;22;0;0;7552;10,5167;;S;;;Norrlot, Sweden Chicago, IL
-3;0;Dakic, Mr. Branko;male;19;0;0;349228;10,1708;;S;;;Austria
-3;1;"Daly, Miss. Margaret Marcella ""Maggie""";female;30;0;0;382650;6,95;;Q;15;;Co Athlone, Ireland New York, NY
-3;1;Daly, Mr. Eugene Patrick;male;29;0;0;382651;7,75;;Q;13 15 B;;Co Athlone, Ireland New York, NY
-3;0;Danbom, Master. Gilbert Sigvard Emanuel;male;0,3333;0;2;347080;14,4;;S;;;Stanton, IA
-3;0;Danbom, Mr. Ernst Gilbert;male;34;1;1;347080;14,4;;S;;197;Stanton, IA
-3;0;Danbom, Mrs. Ernst Gilbert (Anna Sigrid Maria Brogren);female;28;1;1;347080;14,4;;S;;;Stanton, IA
-3;0;Danoff, Mr. Yoto;male;27;0;0;349219;7,8958;;S;;;Bulgaria Chicago, IL
-3;0;Dantcheff, Mr. Ristiu;male;25;0;0;349203;7,8958;;S;;;Bulgaria Chicago, IL
-3;0;Davies, Mr. Alfred J;male;24;2;0;A/4 48871;24,15;;S;;;West Bromwich, England Pontiac, MI
-3;0;Davies, Mr. Evan;male;22;0;0;SC/A4 23568;8,05;;S;;;
-3;0;Davies, Mr. John Samuel;male;21;2;0;A/4 48871;24,15;;S;;;West Bromwich, England Pontiac, MI
-3;0;Davies, Mr. Joseph;male;17;2;0;A/4 48873;8,05;;S;;;West Bromwich, England Pontiac, MI
-3;0;Davison, Mr. Thomas Henry;male;;1;0;386525;16,1;;S;;;Liverpool, England Bedford, OH
-3;1;Davison, Mrs. Thomas Henry (Mary E Finck);female;;1;0;386525;16,1;;S;16;;Liverpool, England Bedford, OH
-3;1;de Messemaeker, Mr. Guillaume Joseph;male;36,5;1;0;345572;17,4;;S;15;;Tampico, MT
-3;1;de Messemaeker, Mrs. Guillaume Joseph (Emma);female;36;1;0;345572;17,4;;S;13;;Tampico, MT
-3;1;de Mulder, Mr. Theodore;male;30;0;0;345774;9,5;;S;11;;Belgium Detroit, MI
-3;0;de Pelsmaeker, Mr. Alfons;male;16;0;0;345778;9,5;;S;;;
-3;1;Dean, Master. Bertram Vere;male;1;1;2;C.A. 2315;20,575;;S;10;;Devon, England Wichita, KS
-3;1;"Dean, Miss. Elizabeth Gladys ""Millvina""";female;0,1667;1;2;C.A. 2315;20,575;;S;10;;Devon, England Wichita, KS
-3;0;Dean, Mr. Bertram Frank;male;26;1;2;C.A. 2315;20,575;;S;;;Devon, England Wichita, KS
-3;1;Dean, Mrs. Bertram (Eva Georgetta Light);female;33;1;2;C.A. 2315;20,575;;S;10;;Devon, England Wichita, KS
-3;0;Delalic, Mr. Redjo;male;25;0;0;349250;7,8958;;S;;;
-3;0;Demetri, Mr. Marinko;male;;0;0;349238;7,8958;;S;;;
-3;0;Denkoff, Mr. Mitto;male;;0;0;349225;7,8958;;S;;;Bulgaria Coon Rapids, IA
-3;0;Dennis, Mr. Samuel;male;22;0;0;A/5 21172;7,25;;S;;;
-3;0;Dennis, Mr. William;male;36;0;0;A/5 21175;7,25;;S;;;
-3;1;Devaney, Miss. Margaret Delia;female;19;0;0;330958;7,8792;;Q;C;;Kilmacowen, Co Sligo, Ireland New York, NY
-3;0;Dika, Mr. Mirko;male;17;0;0;349232;7,8958;;S;;;
-3;0;Dimic, Mr. Jovan;male;42;0;0;315088;8,6625;;S;;;
-3;0;Dintcheff, Mr. Valtcho;male;43;0;0;349226;7,8958;;S;;;
-3;0;Doharr, Mr. Tannous;male;;0;0;2686;7,2292;;C;;;
-3;0;Dooley, Mr. Patrick;male;32;0;0;370376;7,75;;Q;;;Ireland New York, NY
-3;1;Dorking, Mr. Edward Arthur;male;19;0;0;A/5. 10482;8,05;;S;B;;England Oglesby, IL
-3;1;Dowdell, Miss. Elizabeth;female;30;0;0;364516;12,475;;S;13;;Union Hill, NJ
-3;0;Doyle, Miss. Elizabeth;female;24;0;0;368702;7,75;;Q;;;Ireland New York, NY
-3;1;Drapkin, Miss. Jennie;female;23;0;0;SOTON/OQ 392083;8,05;;S;;;London New York, NY
-3;0;Drazenoic, Mr. Jozef;male;33;0;0;349241;7,8958;;C;;51;Austria Niagara Falls, NY
-3;0;Duane, Mr. Frank;male;65;0;0;336439;7,75;;Q;;;
-3;1;Duquemin, Mr. Joseph;male;24;0;0;S.O./P.P. 752;7,55;;S;D;;England Albion, NY
-3;0;Dyker, Mr. Adolf Fredrik;male;23;1;0;347072;13,9;;S;;;West Haven, CT
-3;1;Dyker, Mrs. Adolf Fredrik (Anna Elisabeth Judith Andersson);female;22;1;0;347072;13,9;;S;16;;West Haven, CT
-3;0;Edvardsson, Mr. Gustaf Hjalmar;male;18;0;0;349912;7,775;;S;;;Tofta, Sweden Joliet, IL
-3;0;Eklund, Mr. Hans Linus;male;16;0;0;347074;7,775;;S;;;Karberg, Sweden Jerome Junction, AZ
-3;0;Ekstrom, Mr. Johan;male;45;0;0;347061;6,975;;S;;;Effington Rut, SD
-3;0;Elias, Mr. Dibo;male;;0;0;2674;7,225;;C;;;
-3;0;Elias, Mr. Joseph;male;39;0;2;2675;7,2292;;C;;;Syria Ottawa, ON
-3;0;Elias, Mr. Joseph Jr;male;17;1;1;2690;7,2292;;C;;;
-3;0;Elias, Mr. Tannous;male;15;1;1;2695;7,2292;;C;;;Syria
-3;0;Elsbury, Mr. William James;male;47;0;0;A/5 3902;7,25;;S;;;Illinois, USA
-3;1;Emanuel, Miss. Virginia Ethel;female;5;0;0;364516;12,475;;S;13;;New York, NY
-3;0;Emir, Mr. Farred Chehab;male;;0;0;2631;7,225;;C;;;
-3;0;Everett, Mr. Thomas James;male;40,5;0;0;C.A. 6212;15,1;;S;;187;
-3;0;Farrell, Mr. James;male;40,5;0;0;367232;7,75;;Q;;68;Aughnacliff, Co Longford, Ireland New York, NY
-3;1;Finoli, Mr. Luigi;male;;0;0;SOTON/O.Q. 3101308;7,05;;S;15;;Italy Philadelphia, PA
-3;0;Fischer, Mr. Eberhard Thelander;male;18;0;0;350036;7,7958;;S;;;
-3;0;Fleming, Miss. Honora;female;;0;0;364859;7,75;;Q;;;
-3;0;Flynn, Mr. James;male;;0;0;364851;7,75;;Q;;;
-3;0;Flynn, Mr. John;male;;0;0;368323;6,95;;Q;;;
-3;0;Foley, Mr. Joseph;male;26;0;0;330910;7,8792;;Q;;;Ireland Chicago, IL
-3;0;Foley, Mr. William;male;;0;0;365235;7,75;;Q;;;Ireland
-3;1;Foo, Mr. Choong;male;;0;0;1601;56,4958;;S;13;;Hong Kong New York, NY
-3;0;"Ford, Miss. Doolina Margaret ""Daisy""";female;21;2;2;W./C. 6608;34,375;;S;;;Rotherfield, Sussex, England Essex Co, MA
-3;0;"Ford, Miss. Robina Maggie ""Ruby""";female;9;2;2;W./C. 6608;34,375;;S;;;Rotherfield, Sussex, England Essex Co, MA
-3;0;Ford, Mr. Arthur;male;;0;0;A/5 1478;8,05;;S;;;Bridgwater, Somerset, England
-3;0;Ford, Mr. Edward Watson;male;18;2;2;W./C. 6608;34,375;;S;;;Rotherfield, Sussex, England Essex Co, MA
-3;0;Ford, Mr. William Neal;male;16;1;3;W./C. 6608;34,375;;S;;;Rotherfield, Sussex, England Essex Co, MA
-3;0;Ford, Mrs. Edward (Margaret Ann Watson);female;48;1;3;W./C. 6608;34,375;;S;;;Rotherfield, Sussex, England Essex Co, MA
-3;0;Fox, Mr. Patrick;male;;0;0;368573;7,75;;Q;;;Ireland New York, NY
-3;0;Franklin, Mr. Charles (Charles Fardon);male;;0;0;SOTON/O.Q. 3101314;7,25;;S;;;
-3;0;Gallagher, Mr. Martin;male;25;0;0;36864;7,7417;;Q;;;New York, NY
-3;0;Garfirth, Mr. John;male;;0;0;358585;14,5;;S;;;
-3;0;Gheorgheff, Mr. Stanio;male;;0;0;349254;7,8958;;C;;;
-3;0;Gilinski, Mr. Eliezer;male;22;0;0;14973;8,05;;S;;47;
-3;1;"Gilnagh, Miss. Katherine ""Katie""";female;16;0;0;35851;7,7333;;Q;16;;Co Longford, Ireland New York, NY
-3;1;Glynn, Miss. Mary Agatha;female;;0;0;335677;7,75;;Q;13;;Co Clare, Ireland Washington, DC
-3;1;"Goldsmith, Master. Frank John William ""Frankie""";male;9;0;2;363291;20,525;;S;C D;;Strood, Kent, England Detroit, MI
-3;0;Goldsmith, Mr. Frank John;male;33;1;1;363291;20,525;;S;;;Strood, Kent, England Detroit, MI
-3;0;Goldsmith, Mr. Nathan;male;41;0;0;SOTON/O.Q. 3101263;7,85;;S;;;Philadelphia, PA
-3;1;Goldsmith, Mrs. Frank John (Emily Alice Brown);female;31;1;1;363291;20,525;;S;C D;;Strood, Kent, England Detroit, MI
-3;0;Goncalves, Mr. Manuel Estanslas;male;38;0;0;SOTON/O.Q. 3101306;7,05;;S;;;Portugal
-3;0;Goodwin, Master. Harold Victor;male;9;5;2;CA 2144;46,9;;S;;;Wiltshire, England Niagara Falls, NY
-3;0;Goodwin, Master. Sidney Leonard;male;1;5;2;CA 2144;46,9;;S;;;Wiltshire, England Niagara Falls, NY
-3;0;Goodwin, Master. William Frederick;male;11;5;2;CA 2144;46,9;;S;;;Wiltshire, England Niagara Falls, NY
-3;0;Goodwin, Miss. Jessie Allis;female;10;5;2;CA 2144;46,9;;S;;;Wiltshire, England Niagara Falls, NY
-3;0;Goodwin, Miss. Lillian Amy;female;16;5;2;CA 2144;46,9;;S;;;Wiltshire, England Niagara Falls, NY
-3;0;Goodwin, Mr. Charles Edward;male;14;5;2;CA 2144;46,9;;S;;;Wiltshire, England Niagara Falls, NY
-3;0;Goodwin, Mr. Charles Frederick;male;40;1;6;CA 2144;46,9;;S;;;Wiltshire, England Niagara Falls, NY
-3;0;Goodwin, Mrs. Frederick (Augusta Tyler);female;43;1;6;CA 2144;46,9;;S;;;Wiltshire, England Niagara Falls, NY
-3;0;Green, Mr. George Henry;male;51;0;0;21440;8,05;;S;;;Dorking, Surrey, England
-3;0;Gronnestad, Mr. Daniel Danielsen;male;32;0;0;8471;8,3625;;S;;;Foresvik, Norway Portland, ND
-3;0;Guest, Mr. Robert;male;;0;0;376563;8,05;;S;;;
-3;0;Gustafsson, Mr. Alfred Ossian;male;20;0;0;7534;9,8458;;S;;;Waukegan, Chicago, IL
-3;0;Gustafsson, Mr. Anders Vilhelm;male;37;2;0;3101276;7,925;;S;;98;Ruotsinphytaa, Finland New York, NY
-3;0;Gustafsson, Mr. Johan Birger;male;28;2;0;3101277;7,925;;S;;;Ruotsinphytaa, Finland New York, NY
-3;0;Gustafsson, Mr. Karl Gideon;male;19;0;0;347069;7,775;;S;;;Myren, Sweden New York, NY
-3;0;Haas, Miss. Aloisia;female;24;0;0;349236;8,85;;S;;;
-3;0;Hagardon, Miss. Kate;female;17;0;0;AQ/3. 30631;7,7333;;Q;;;
-3;0;Hagland, Mr. Ingvald Olai Olsen;male;;1;0;65303;19,9667;;S;;;
-3;0;Hagland, Mr. Konrad Mathias Reiersen;male;;1;0;65304;19,9667;;S;;;
-3;0;Hakkarainen, Mr. Pekka Pietari;male;28;1;0;STON/O2. 3101279;15,85;;S;;;
-3;1;Hakkarainen, Mrs. Pekka Pietari (Elin Matilda Dolck);female;24;1;0;STON/O2. 3101279;15,85;;S;15;;
-3;0;Hampe, Mr. Leon;male;20;0;0;345769;9,5;;S;;;
-3;0;Hanna, Mr. Mansour;male;23,5;0;0;2693;7,2292;;C;;188;
-3;0;Hansen, Mr. Claus Peter;male;41;2;0;350026;14,1083;;S;;;
-3;0;Hansen, Mr. Henrik Juul;male;26;1;0;350025;7,8542;;S;;;
-3;0;Hansen, Mr. Henry Damsgaard;male;21;0;0;350029;7,8542;;S;;69;
-3;1;Hansen, Mrs. Claus Peter (Jennie L Howard);female;45;1;0;350026;14,1083;;S;11;;
-3;0;Harknett, Miss. Alice Phoebe;female;;0;0;W./C. 6609;7,55;;S;;;
-3;0;Harmer, Mr. Abraham (David Lishin);male;25;0;0;374887;7,25;;S;B;;
-3;0;Hart, Mr. Henry;male;;0;0;394140;6,8583;;Q;;;
-3;0;Hassan, Mr. Houssein G N;male;11;0;0;2699;18,7875;;C;;;
-3;1;"Healy, Miss. Hanora ""Nora""";female;;0;0;370375;7,75;;Q;16;;
-3;1;Hedman, Mr. Oskar Arvid;male;27;0;0;347089;6,975;;S;15;;
-3;1;Hee, Mr. Ling;male;;0;0;1601;56,4958;;S;C;;
-3;0;"Hegarty, Miss. Hanora ""Nora""";female;18;0;0;365226;6,75;;Q;;;
-3;1;Heikkinen, Miss. Laina;female;26;0;0;STON/O2. 3101282;7,925;;S;;;
-3;0;Heininen, Miss. Wendla Maria;female;23;0;0;STON/O2. 3101290;7,925;;S;;;
-3;1;Hellstrom, Miss. Hilda Maria;female;22;0;0;7548;8,9625;;S;C;;
-3;0;Hendekovic, Mr. Ignjac;male;28;0;0;349243;7,8958;;S;;306;
-3;0;Henriksson, Miss. Jenny Lovisa;female;28;0;0;347086;7,775;;S;;;
-3;0;Henry, Miss. Delia;female;;0;0;382649;7,75;;Q;;;
-3;1;Hirvonen, Miss. Hildur E;female;2;0;1;3101298;12,2875;;S;15;;
-3;1;Hirvonen, Mrs. Alexander (Helga E Lindqvist);female;22;1;1;3101298;12,2875;;S;15;;
-3;0;Holm, Mr. John Fredrik Alexander;male;43;0;0;C 7075;6,45;;S;;;
-3;0;Holthen, Mr. Johan Martin;male;28;0;0;C 4001;22,525;;S;;;
-3;1;Honkanen, Miss. Eliina;female;27;0;0;STON/O2. 3101283;7,925;;S;;;
-3;0;Horgan, Mr. John;male;;0;0;370377;7,75;;Q;;;
-3;1;Howard, Miss. May Elizabeth;female;;0;0;A. 2. 39186;8,05;;S;C;;
-3;0;Humblen, Mr. Adolf Mathias Nicolai Olsen;male;42;0;0;348121;7,65;F G63;S;;120;
-3;1;Hyman, Mr. Abraham;male;;0;0;3470;7,8875;;S;C;;
-3;0;Ibrahim Shawah, Mr. Yousseff;male;30;0;0;2685;7,2292;;C;;;
-3;0;Ilieff, Mr. Ylio;male;;0;0;349220;7,8958;;S;;;
-3;0;Ilmakangas, Miss. Ida Livija;female;27;1;0;STON/O2. 3101270;7,925;;S;;;
-3;0;Ilmakangas, Miss. Pieta Sofia;female;25;1;0;STON/O2. 3101271;7,925;;S;;;
-3;0;Ivanoff, Mr. Kanio;male;;0;0;349201;7,8958;;S;;;
-3;1;Jalsevac, Mr. Ivan;male;29;0;0;349240;7,8958;;C;15;;
-3;1;Jansson, Mr. Carl Olof;male;21;0;0;350034;7,7958;;S;A;;
-3;0;Jardin, Mr. Jose Neto;male;;0;0;SOTON/O.Q. 3101305;7,05;;S;;;
-3;0;Jensen, Mr. Hans Peder;male;20;0;0;350050;7,8542;;S;;;
-3;0;Jensen, Mr. Niels Peder;male;48;0;0;350047;7,8542;;S;;;
-3;0;Jensen, Mr. Svend Lauritz;male;17;1;0;350048;7,0542;;S;;;
-3;1;Jermyn, Miss. Annie;female;;0;0;14313;7,75;;Q;D;;
-3;1;Johannesen-Bratthammer, Mr. Bernt;male;;0;0;65306;8,1125;;S;13;;
-3;0;Johanson, Mr. Jakob Alfred;male;34;0;0;3101264;6,4958;;S;;143;
-3;1;Johansson Palmquist, Mr. Oskar Leander;male;26;0;0;347070;7,775;;S;15;;
-3;0;Johansson, Mr. Erik;male;22;0;0;350052;7,7958;;S;;156;
-3;0;Johansson, Mr. Gustaf Joel;male;33;0;0;7540;8,6542;;S;;285;
-3;0;Johansson, Mr. Karl Johan;male;31;0;0;347063;7,775;;S;;;
-3;0;Johansson, Mr. Nils;male;29;0;0;347467;7,8542;;S;;;
-3;1;Johnson, Master. Harold Theodor;male;4;1;1;347742;11,1333;;S;15;;
-3;1;Johnson, Miss. Eleanor Ileen;female;1;1;1;347742;11,1333;;S;15;;
-3;0;Johnson, Mr. Alfred;male;49;0;0;LINE;0;;S;;;
-3;0;Johnson, Mr. Malkolm Joackim;male;33;0;0;347062;7,775;;S;;37;
-3;0;Johnson, Mr. William Cahoone Jr;male;19;0;0;LINE;0;;S;;;
-3;1;Johnson, Mrs. Oscar W (Elisabeth Vilhelmina Berg);female;27;0;2;347742;11,1333;;S;15;;
-3;0;"Johnston, Master. William Arthur ""Willie""";male;;1;2;W./C. 6607;23,45;;S;;;
-3;0;"Johnston, Miss. Catherine Helen ""Carrie""";female;;1;2;W./C. 6607;23,45;;S;;;
-3;0;Johnston, Mr. Andrew G;male;;1;2;W./C. 6607;23,45;;S;;;
-3;0;"Johnston, Mrs. Andrew G (Elizabeth ""Lily"" Watson)";female;;1;2;W./C. 6607;23,45;;S;;;
-3;0;Jonkoff, Mr. Lalio;male;23;0;0;349204;7,8958;;S;;;
-3;1;Jonsson, Mr. Carl;male;32;0;0;350417;7,8542;;S;15;;
-3;0;Jonsson, Mr. Nils Hilding;male;27;0;0;350408;7,8542;;S;;;
-3;0;Jussila, Miss. Katriina;female;20;1;0;4136;9,825;;S;;;
-3;0;Jussila, Miss. Mari Aina;female;21;1;0;4137;9,825;;S;;;
-3;1;Jussila, Mr. Eiriik;male;32;0;0;STON/O 2. 3101286;7,925;;S;15;;
-3;0;Kallio, Mr. Nikolai Erland;male;17;0;0;STON/O 2. 3101274;7,125;;S;;;
-3;0;Kalvik, Mr. Johannes Halvorsen;male;21;0;0;8475;8,4333;;S;;;
-3;0;Karaic, Mr. Milan;male;30;0;0;349246;7,8958;;S;;;
-3;1;Karlsson, Mr. Einar Gervasius;male;21;0;0;350053;7,7958;;S;13;;
-3;0;Karlsson, Mr. Julius Konrad Eugen;male;33;0;0;347465;7,8542;;S;;;
-3;0;Karlsson, Mr. Nils August;male;22;0;0;350060;7,5208;;S;;;
-3;1;Karun, Miss. Manca;female;4;0;1;349256;13,4167;;C;15;;
-3;1;Karun, Mr. Franz;male;39;0;1;349256;13,4167;;C;15;;
-3;0;Kassem, Mr. Fared;male;;0;0;2700;7,2292;;C;;;
-3;0;"Katavelas, Mr. Vassilios (""Catavelas Vassilios"")";male;18,5;0;0;2682;7,2292;;C;;58;
-3;0;"Keane, Mr. Andrew ""Andy""";male;;0;0;12460;7,75;;Q;;;
-3;0;Keefe, Mr. Arthur;male;;0;0;323592;7,25;;S;A;;
-3;1;"Kelly, Miss. Anna Katherine ""Annie Kate""";female;;0;0;9234;7,75;;Q;16;;
-3;1;Kelly, Miss. Mary;female;;0;0;14312;7,75;;Q;D;;
-3;0;Kelly, Mr. James;male;34,5;0;0;330911;7,8292;;Q;;70;
-3;0;Kelly, Mr. James;male;44;0;0;363592;8,05;;S;;;
-3;1;Kennedy, Mr. John;male;;0;0;368783;7,75;;Q;;;
-3;0;Khalil, Mr. Betros;male;;1;0;2660;14,4542;;C;;;
-3;0;"Khalil, Mrs. Betros (Zahie ""Maria"" Elias)";female;;1;0;2660;14,4542;;C;;;
-3;0;Kiernan, Mr. John;male;;1;0;367227;7,75;;Q;;;
-3;0;Kiernan, Mr. Philip;male;;1;0;367229;7,75;;Q;;;
-3;0;Kilgannon, Mr. Thomas J;male;;0;0;36865;7,7375;;Q;;;
-3;0;Kink, Miss. Maria;female;22;2;0;315152;8,6625;;S;;;
-3;0;Kink, Mr. Vincenz;male;26;2;0;315151;8,6625;;S;;;
-3;1;Kink-Heilmann, Miss. Luise Gretchen;female;4;0;2;315153;22,025;;S;2;;
-3;1;Kink-Heilmann, Mr. Anton;male;29;3;1;315153;22,025;;S;2;;
-3;1;Kink-Heilmann, Mrs. Anton (Luise Heilmann);female;26;1;1;315153;22,025;;S;2;;
-3;0;Klasen, Miss. Gertrud Emilia;female;1;1;1;350405;12,1833;;S;;;
-3;0;Klasen, Mr. Klas Albin;male;18;1;1;350404;7,8542;;S;;;
-3;0;Klasen, Mrs. (Hulda Kristina Eugenia Lofqvist);female;36;0;2;350405;12,1833;;S;;;
-3;0;Kraeff, Mr. Theodor;male;;0;0;349253;7,8958;;C;;;
-3;1;Krekorian, Mr. Neshan;male;25;0;0;2654;7,2292;F E57;C;10;;
-3;0;Lahoud, Mr. Sarkis;male;;0;0;2624;7,225;;C;;;
-3;0;Laitinen, Miss. Kristina Sofia;female;37;0;0;4135;9,5875;;S;;;
-3;0;Laleff, Mr. Kristo;male;;0;0;349217;7,8958;;S;;;
-3;1;Lam, Mr. Ali;male;;0;0;1601;56,4958;;S;C;;
-3;0;Lam, Mr. Len;male;;0;0;1601;56,4958;;S;;;
-3;1;Landergren, Miss. Aurora Adelia;female;22;0;0;C 7077;7,25;;S;13;;
-3;0;Lane, Mr. Patrick;male;;0;0;7935;7,75;;Q;;;
-3;1;Lang, Mr. Fang;male;26;0;0;1601;56,4958;;S;14;;
-3;0;Larsson, Mr. August Viktor;male;29;0;0;7545;9,4833;;S;;;
-3;0;Larsson, Mr. Bengt Edvin;male;29;0;0;347067;7,775;;S;;;
-3;0;Larsson-Rondberg, Mr. Edvard A;male;22;0;0;347065;7,775;;S;;;
-3;1;"Leeni, Mr. Fahim (""Philip Zenni"")";male;22;0;0;2620;7,225;;C;6;;
-3;0;Lefebre, Master. Henry Forbes;male;;3;1;4133;25,4667;;S;;;
-3;0;Lefebre, Miss. Ida;female;;3;1;4133;25,4667;;S;;;
-3;0;Lefebre, Miss. Jeannie;female;;3;1;4133;25,4667;;S;;;
-3;0;Lefebre, Miss. Mathilde;female;;3;1;4133;25,4667;;S;;;
-3;0;Lefebre, Mrs. Frank (Frances);female;;0;4;4133;25,4667;;S;;;
-3;0;Leinonen, Mr. Antti Gustaf;male;32;0;0;STON/O 2. 3101292;7,925;;S;;;
-3;0;Lemberopolous, Mr. Peter L;male;34,5;0;0;2683;6,4375;;C;;196;
-3;0;Lennon, Miss. Mary;female;;1;0;370371;15,5;;Q;;;
-3;0;Lennon, Mr. Denis;male;;1;0;370371;15,5;;Q;;;
-3;0;Leonard, Mr. Lionel;male;36;0;0;LINE;0;;S;;;
-3;0;Lester, Mr. James;male;39;0;0;A/4 48871;24,15;;S;;;
-3;0;Lievens, Mr. Rene Aime;male;24;0;0;345781;9,5;;S;;;
-3;0;Lindahl, Miss. Agda Thorilda Viktoria;female;25;0;0;347071;7,775;;S;;;
-3;0;Lindblom, Miss. Augusta Charlotta;female;45;0;0;347073;7,75;;S;;;
-3;0;Lindell, Mr. Edvard Bengtsson;male;36;1;0;349910;15,55;;S;A;;
-3;0;Lindell, Mrs. Edvard Bengtsson (Elin Gerda Persson);female;30;1;0;349910;15,55;;S;A;;
-3;1;Lindqvist, Mr. Eino William;male;20;1;0;STON/O 2. 3101285;7,925;;S;15;;
-3;0;Linehan, Mr. Michael;male;;0;0;330971;7,8792;;Q;;;
-3;0;Ling, Mr. Lee;male;28;0;0;1601;56,4958;;S;;;
-3;0;Lithman, Mr. Simon;male;;0;0;S.O./P.P. 251;7,55;;S;;;
-3;0;Lobb, Mr. William Arthur;male;30;1;0;A/5. 3336;16,1;;S;;;
-3;0;Lobb, Mrs. William Arthur (Cordelia K Stanlick);female;26;1;0;A/5. 3336;16,1;;S;;;
-3;0;Lockyer, Mr. Edward;male;;0;0;1222;7,8792;;S;;153;
-3;0;"Lovell, Mr. John Hall (""Henry"")";male;20,5;0;0;A/5 21173;7,25;;S;;;
-3;1;Lulic, Mr. Nikola;male;27;0;0;315098;8,6625;;S;15;;
-3;0;Lundahl, Mr. Johan Svensson;male;51;0;0;347743;7,0542;;S;;;
-3;1;Lundin, Miss. Olga Elida;female;23;0;0;347469;7,8542;;S;10;;
-3;1;Lundstrom, Mr. Thure Edvin;male;32;0;0;350403;7,5792;;S;15;;
-3;0;Lyntakoff, Mr. Stanko;male;;0;0;349235;7,8958;;S;;;
-3;0;MacKay, Mr. George William;male;;0;0;C.A. 42795;7,55;;S;;;
-3;1;"Madigan, Miss. Margaret ""Maggie""";female;;0;0;370370;7,75;;Q;15;;
-3;1;Madsen, Mr. Fridtjof Arne;male;24;0;0;C 17369;7,1417;;S;13;;
-3;0;Maenpaa, Mr. Matti Alexanteri;male;22;0;0;STON/O 2. 3101275;7,125;;S;;;
-3;0;Mahon, Miss. Bridget Delia;female;;0;0;330924;7,8792;;Q;;;
-3;0;Mahon, Mr. John;male;;0;0;AQ/4 3130;7,75;;Q;;;
-3;0;Maisner, Mr. Simon;male;;0;0;A/S 2816;8,05;;S;;;
-3;0;Makinen, Mr. Kalle Edvard;male;29;0;0;STON/O 2. 3101268;7,925;;S;;;
-3;1;Mamee, Mr. Hanna;male;;0;0;2677;7,2292;;C;15;;
-3;0;Mangan, Miss. Mary;female;30,5;0;0;364850;7,75;;Q;;61;
-3;1;Mannion, Miss. Margareth;female;;0;0;36866;7,7375;;Q;16;;
-3;0;Mardirosian, Mr. Sarkis;male;;0;0;2655;7,2292;F E46;C;;;
-3;0;Markoff, Mr. Marin;male;35;0;0;349213;7,8958;;C;;;
-3;0;Markun, Mr. Johann;male;33;0;0;349257;7,8958;;S;;;
-3;1;Masselmani, Mrs. Fatima;female;;0;0;2649;7,225;;C;C;;
-3;0;Matinoff, Mr. Nicola;male;;0;0;349255;7,8958;;C;;;
-3;1;"McCarthy, Miss. Catherine ""Katie""";female;;0;0;383123;7,75;;Q;15 16;;
-3;1;McCormack, Mr. Thomas Joseph;male;;0;0;367228;7,75;;Q;;;
-3;1;McCoy, Miss. Agnes;female;;2;0;367226;23,25;;Q;16;;
-3;1;McCoy, Miss. Alicia;female;;2;0;367226;23,25;;Q;16;;
-3;1;McCoy, Mr. Bernard;male;;2;0;367226;23,25;;Q;16;;
-3;1;McDermott, Miss. Brigdet Delia;female;;0;0;330932;7,7875;;Q;13;;
-3;0;McEvoy, Mr. Michael;male;;0;0;36568;15,5;;Q;;;
-3;1;McGovern, Miss. Mary;female;;0;0;330931;7,8792;;Q;13;;
-3;1;"McGowan, Miss. Anna ""Annie""";female;15;0;0;330923;8,0292;;Q;;;
-3;0;McGowan, Miss. Katherine;female;35;0;0;9232;7,75;;Q;;;
-3;0;McMahon, Mr. Martin;male;;0;0;370372;7,75;;Q;;;
-3;0;McNamee, Mr. Neal;male;24;1;0;376566;16,1;;S;;;
-3;0;McNamee, Mrs. Neal (Eileen O'Leary);female;19;1;0;376566;16,1;;S;;53;
-3;0;McNeill, Miss. Bridget;female;;0;0;370368;7,75;;Q;;;
-3;0;Meanwell, Miss. (Marion Ogden);female;;0;0;SOTON/O.Q. 392087;8,05;;S;;;
-3;0;Meek, Mrs. Thomas (Annie Louise Rowley);female;;0;0;343095;8,05;;S;;;
-3;0;Meo, Mr. Alfonzo;male;55,5;0;0;A.5. 11206;8,05;;S;;201;
-3;0;Mernagh, Mr. Robert;male;;0;0;368703;7,75;;Q;;;
-3;1;Midtsjo, Mr. Karl Albert;male;21;0;0;345501;7,775;;S;15;;
-3;0;Miles, Mr. Frank;male;;0;0;359306;8,05;;S;;;
-3;0;Mineff, Mr. Ivan;male;24;0;0;349233;7,8958;;S;;;
-3;0;Minkoff, Mr. Lazar;male;21;0;0;349211;7,8958;;S;;;
-3;0;Mionoff, Mr. Stoytcho;male;28;0;0;349207;7,8958;;S;;;
-3;0;Mitkoff, Mr. Mito;male;;0;0;349221;7,8958;;S;;;
-3;1;"Mockler, Miss. Helen Mary ""Ellie""";female;;0;0;330980;7,8792;;Q;16;;
-3;0;Moen, Mr. Sigurd Hansen;male;25;0;0;348123;7,65;F G73;S;;309;
-3;1;Moor, Master. Meier;male;6;0;1;392096;12,475;E121;S;14;;
-3;1;Moor, Mrs. (Beila);female;27;0;1;392096;12,475;E121;S;14;;
-3;0;Moore, Mr. Leonard Charles;male;;0;0;A4. 54510;8,05;;S;;;
-3;1;Moran, Miss. Bertha;female;;1;0;371110;24,15;;Q;16;;
-3;0;Moran, Mr. Daniel J;male;;1;0;371110;24,15;;Q;;;
-3;0;Moran, Mr. James;male;;0;0;330877;8,4583;;Q;;;
-3;0;Morley, Mr. William;male;34;0;0;364506;8,05;;S;;;
-3;0;Morrow, Mr. Thomas Rowan;male;;0;0;372622;7,75;;Q;;;
-3;1;Moss, Mr. Albert Johan;male;;0;0;312991;7,775;;S;B;;
-3;1;Moubarek, Master. Gerios;male;;1;1;2661;15,2458;;C;C;;
-3;1;"Moubarek, Master. Halim Gonios (""William George"")";male;;1;1;2661;15,2458;;C;C;;
-3;1;"Moubarek, Mrs. George (Omine ""Amenia"" Alexander)";female;;0;2;2661;15,2458;;C;C;;
-3;1;Moussa, Mrs. (Mantoura Boulos);female;;0;0;2626;7,2292;;C;;;
-3;0;Moutal, Mr. Rahamin Haim;male;;0;0;374746;8,05;;S;;;
-3;1;"Mullens, Miss. Katherine ""Katie""";female;;0;0;35852;7,7333;;Q;16;;
-3;1;Mulvihill, Miss. Bertha E;female;24;0;0;382653;7,75;;Q;15;;
-3;0;Murdlin, Mr. Joseph;male;;0;0;A./5. 3235;8,05;;S;;;
-3;1;"Murphy, Miss. Katherine ""Kate""";female;;1;0;367230;15,5;;Q;16;;
-3;1;Murphy, Miss. Margaret Jane;female;;1;0;367230;15,5;;Q;16;;
-3;1;Murphy, Miss. Nora;female;;0;0;36568;15,5;;Q;16;;
-3;0;Myhrman, Mr. Pehr Fabian Oliver Malkolm;male;18;0;0;347078;7,75;;S;;;
-3;0;Naidenoff, Mr. Penko;male;22;0;0;349206;7,8958;;S;;;
-3;1;"Najib, Miss. Adele Kiamie ""Jane""";female;15;0;0;2667;7,225;;C;C;;
-3;1;"Nakid, Miss. Maria (""Mary"")";female;1;0;2;2653;15,7417;;C;C;;
-3;1;Nakid, Mr. Sahid;male;20;1;1;2653;15,7417;;C;C;;
-3;1;"Nakid, Mrs. Said (Waika ""Mary"" Mowad)";female;19;1;1;2653;15,7417;;C;C;;
-3;0;Nancarrow, Mr. William Henry;male;33;0;0;A./5. 3338;8,05;;S;;;
-3;0;Nankoff, Mr. Minko;male;;0;0;349218;7,8958;;S;;;
-3;0;Nasr, Mr. Mustafa;male;;0;0;2652;7,2292;;C;;;
-3;0;Naughton, Miss. Hannah;female;;0;0;365237;7,75;;Q;;;
-3;0;Nenkoff, Mr. Christo;male;;0;0;349234;7,8958;;S;;;
-3;1;Nicola-Yarred, Master. Elias;male;12;1;0;2651;11,2417;;C;C;;
-3;1;Nicola-Yarred, Miss. Jamila;female;14;1;0;2651;11,2417;;C;C;;
-3;0;Nieminen, Miss. Manta Josefina;female;29;0;0;3101297;7,925;;S;;;
-3;0;Niklasson, Mr. Samuel;male;28;0;0;363611;8,05;;S;;;
-3;1;Nilsson, Miss. Berta Olivia;female;18;0;0;347066;7,775;;S;D;;
-3;1;Nilsson, Miss. Helmina Josefina;female;26;0;0;347470;7,8542;;S;13;;
-3;0;Nilsson, Mr. August Ferdinand;male;21;0;0;350410;7,8542;;S;;;
-3;0;Nirva, Mr. Iisakki Antino Aijo;male;41;0;0;SOTON/O2 3101272;7,125;;S;;;Finland Sudbury, ON
-3;1;Niskanen, Mr. Juha;male;39;0;0;STON/O 2. 3101289;7,925;;S;9;;
-3;0;Nosworthy, Mr. Richard Cater;male;21;0;0;A/4. 39886;7,8;;S;;;
-3;0;Novel, Mr. Mansouer;male;28,5;0;0;2697;7,2292;;C;;181;
-3;1;Nysten, Miss. Anna Sofia;female;22;0;0;347081;7,75;;S;13;;
-3;0;Nysveen, Mr. Johan Hansen;male;61;0;0;345364;6,2375;;S;;;
-3;0;O'Brien, Mr. Thomas;male;;1;0;370365;15,5;;Q;;;
-3;0;O'Brien, Mr. Timothy;male;;0;0;330979;7,8292;;Q;;;
-3;1;"O'Brien, Mrs. Thomas (Johanna ""Hannah"" Godfrey)";female;;1;0;370365;15,5;;Q;;;
-3;0;O'Connell, Mr. Patrick D;male;;0;0;334912;7,7333;;Q;;;
-3;0;O'Connor, Mr. Maurice;male;;0;0;371060;7,75;;Q;;;
-3;0;O'Connor, Mr. Patrick;male;;0;0;366713;7,75;;Q;;;
-3;0;Odahl, Mr. Nils Martin;male;23;0;0;7267;9,225;;S;;;
-3;0;O'Donoghue, Ms. Bridget;female;;0;0;364856;7,75;;Q;;;
-3;1;O'Driscoll, Miss. Bridget;female;;0;0;14311;7,75;;Q;D;;
-3;1;"O'Dwyer, Miss. Ellen ""Nellie""";female;;0;0;330959;7,8792;;Q;;;
-3;1;Ohman, Miss. Velin;female;22;0;0;347085;7,775;;S;C;;
-3;1;O'Keefe, Mr. Patrick;male;;0;0;368402;7,75;;Q;B;;
-3;1;"O'Leary, Miss. Hanora ""Norah""";female;;0;0;330919;7,8292;;Q;13;;
-3;1;Olsen, Master. Artur Karl;male;9;0;1;C 17368;3,1708;;S;13;;
-3;0;Olsen, Mr. Henry Margido;male;28;0;0;C 4001;22,525;;S;;173;
-3;0;Olsen, Mr. Karl Siegwart Andreas;male;42;0;1;4579;8,4042;;S;;;
-3;0;Olsen, Mr. Ole Martin;male;;0;0;Fa 265302;7,3125;;S;;;
-3;0;Olsson, Miss. Elina;female;31;0;0;350407;7,8542;;S;;;
-3;0;Olsson, Mr. Nils Johan Goransson;male;28;0;0;347464;7,8542;;S;;;
-3;1;Olsson, Mr. Oscar Wilhelm;male;32;0;0;347079;7,775;;S;A;;
-3;0;Olsvigen, Mr. Thor Anderson;male;20;0;0;6563;9,225;;S;;89;Oslo, Norway Cameron, WI
-3;0;Oreskovic, Miss. Jelka;female;23;0;0;315085;8,6625;;S;;;
-3;0;Oreskovic, Miss. Marija;female;20;0;0;315096;8,6625;;S;;;
-3;0;Oreskovic, Mr. Luka;male;20;0;0;315094;8,6625;;S;;;
-3;0;Osen, Mr. Olaf Elon;male;16;0;0;7534;9,2167;;S;;;
-3;1;Osman, Mrs. Mara;female;31;0;0;349244;8,6833;;S;;;
-3;0;O'Sullivan, Miss. Bridget Mary;female;;0;0;330909;7,6292;;Q;;;
-3;0;Palsson, Master. Gosta Leonard;male;2;3;1;349909;21,075;;S;;4;
-3;0;Palsson, Master. Paul Folke;male;6;3;1;349909;21,075;;S;;;
-3;0;Palsson, Miss. Stina Viola;female;3;3;1;349909;21,075;;S;;;
-3;0;Palsson, Miss. Torborg Danira;female;8;3;1;349909;21,075;;S;;;
-3;0;Palsson, Mrs. Nils (Alma Cornelia Berglund);female;29;0;4;349909;21,075;;S;;206;
-3;0;Panula, Master. Eino Viljami;male;1;4;1;3101295;39,6875;;S;;;
-3;0;Panula, Master. Juha Niilo;male;7;4;1;3101295;39,6875;;S;;;
-3;0;Panula, Master. Urho Abraham;male;2;4;1;3101295;39,6875;;S;;;
-3;0;Panula, Mr. Ernesti Arvid;male;16;4;1;3101295;39,6875;;S;

<TRUNCATED>

[24/50] [abbrv] ignite git commit: IGNITE-9818 Fix javadoc for annotation AffinityKeyMapped - Fixes #4927.

Posted by ag...@apache.org.
IGNITE-9818 Fix javadoc for annotation AffinityKeyMapped - Fixes #4927.

Signed-off-by: Dmitriy Pavlov <dp...@apache.org>


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

Branch: refs/heads/ignite-9720
Commit: 6fd6c320ed0f79dc71a54981982361ab8dc30f44
Parents: 04fae6d
Author: Max-Pudov <pu...@gmail.com>
Authored: Mon Nov 26 17:46:56 2018 +0300
Committer: Dmitriy Pavlov <dp...@apache.org>
Committed: Mon Nov 26 17:46:56 2018 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/cache/affinity/AffinityKeyMapped.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6fd6c320/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityKeyMapped.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityKeyMapped.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityKeyMapped.java
index 8b19338..e7e9eba 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityKeyMapped.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityKeyMapped.java
@@ -91,8 +91,8 @@ import java.util.concurrent.Callable;
  * is otherwise known as {@code Collocation Of Computations And Data}. In this case,
  * {@code @AffinityKeyMapped} annotation allows to specify a routing affinity key for a
  * {@link org.apache.ignite.compute.ComputeJob} or any other grid computation, such as {@link Runnable},
- * {@link Callable}, or {@link org.apache.ignite.lang.IgniteClosure}. It should be attached to a method or
- * field that provides affinity key for the computation. Only one annotation per class is allowed.
+ * {@link Callable}, or {@link org.apache.ignite.lang.IgniteClosure}. It should be attached to a field
+ * that provides affinity key for the computation. Only one annotation per class is allowed.
  * Whenever such annotation is detected, then {@link org.apache.ignite.spi.loadbalancing.LoadBalancingSpi}
  * will be bypassed, and computation will be routed to the grid node where the specified affinity key is cached.
  * <p>