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

[16/51] ignite git commit: Get rid of portable flag in Ignite

Get rid of portable flag in Ignite


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

Branch: refs/heads/ignite-5075-pds
Commit: 827b7f61faf65406040f2f66349044cf91638b8c
Parents: f5bbc71
Author: Konstantin Dudkov <kd...@ya.ru>
Authored: Mon May 29 16:00:35 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon May 29 16:01:25 2017 +0300

----------------------------------------------------------------------
 .../cache/query/GridCacheQueryRequest.java      | 14 ++--
 .../BinaryAtomicCacheLocalEntriesSelfTest.java  | 30 +++++++
 .../BinaryTxCacheLocalEntriesSelfTest.java      | 84 ++++++++++++++++++++
 .../BinaryAtomicCacheLocalEntriesSelfTest.java  | 30 -------
 .../BinaryTxCacheLocalEntriesSelfTest.java      | 84 --------------------
 .../IgniteBinaryObjectsTestSuite.java           |  4 +-
 .../h2/twostep/GridReduceQueryExecutor.java     | 10 +--
 7 files changed, 128 insertions(+), 128 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/827b7f61/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
index bc32c60..00ddff8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
@@ -114,7 +114,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
     private boolean all;
 
     /** */
-    private boolean keepPortable;
+    private boolean keepBinary;
 
     /** */
     private UUID subjId;
@@ -195,7 +195,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
         this.incBackups = incBackups;
         this.fields = fields;
         this.all = all;
-        this.keepPortable = keepBinary;
+        this.keepBinary = keepBinary;
         this.subjId = subjId;
         this.taskHash = taskHash;
         this.topVer = topVer;
@@ -265,7 +265,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
         this.incBackups = incBackups;
         this.args = args;
         this.incMeta = incMeta;
-        this.keepPortable = keepBinary;
+        this.keepBinary = keepBinary;
         this.subjId = subjId;
         this.taskHash = taskHash;
         this.topVer = topVer;
@@ -460,7 +460,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
      * @return Whether to keep binary.
      */
     public boolean keepBinary() {
-        return keepPortable;
+        return keepBinary;
     }
 
     /**
@@ -480,7 +480,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
     /**
      * @return partition.
      */
-    public int partition() {
+    @Override public int partition() {
         return part;
     }
 
@@ -560,7 +560,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
                 writer.incrementState();
 
             case 13:
-                if (!writer.writeBoolean("keepPortable", keepPortable))
+                if (!writer.writeBoolean("keepBinary", keepBinary))
                     return false;
 
                 writer.incrementState();
@@ -716,7 +716,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
                 reader.incrementState();
 
             case 13:
-                keepPortable = reader.readBoolean("keepPortable");
+                keepBinary = reader.readBoolean("keepBinary");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/827b7f61/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryAtomicCacheLocalEntriesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryAtomicCacheLocalEntriesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryAtomicCacheLocalEntriesSelfTest.java
new file mode 100644
index 0000000..d68b236
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryAtomicCacheLocalEntriesSelfTest.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.cache.binary;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+
+/**
+ *
+ */
+public class BinaryAtomicCacheLocalEntriesSelfTest extends BinaryTxCacheLocalEntriesSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.ATOMIC;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/827b7f61/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryTxCacheLocalEntriesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryTxCacheLocalEntriesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryTxCacheLocalEntriesSelfTest.java
new file mode 100644
index 0000000..3528161
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryTxCacheLocalEntriesSelfTest.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.binary;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.cache.CachePeekMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
+
+/**
+ *
+ */
+public class BinaryTxCacheLocalEntriesSelfTest extends GridCacheAbstractSelfTest {
+    /** */
+    private static final String FIELD = "user-name";
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 2;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception {
+        CacheConfiguration ccfg = super.cacheConfiguration(igniteInstanceName);
+
+        ccfg.setStoreKeepBinary(true);
+
+        return ccfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLocalEntries() throws Exception {
+        IgniteCache<Integer, BinaryObject> cache = grid(0).cache(DEFAULT_CACHE_NAME).withKeepBinary();
+
+        final int ENTRY_CNT = 10;
+
+        for (int i = 0; i < ENTRY_CNT; i++)
+            cache.put(i, userObject("user-" + i));
+
+        assertEquals(ENTRY_CNT, cache.localSize(CachePeekMode.ALL));
+
+        for (int i = 0; i < gridCount(); i++)
+            jcache(i).withKeepBinary().localEntries();
+
+        cache.removeAll();
+    }
+
+    /**
+     * @param userName User name.
+     * @return Binary object.
+     */
+    private BinaryObject userObject(String userName) {
+        return grid(0).binary().builder("orders").setField(FIELD, userName).build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/827b7f61/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/BinaryAtomicCacheLocalEntriesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/BinaryAtomicCacheLocalEntriesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/BinaryAtomicCacheLocalEntriesSelfTest.java
deleted file mode 100644
index 41a4286..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/BinaryAtomicCacheLocalEntriesSelfTest.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one or more
-* contributor license agreements.  See the NOTICE file distributed with
-* this work for additional information regarding copyright ownership.
-* The ASF licenses this file to You under the Apache License, Version 2.0
-* (the "License"); you may not use this file except in compliance with
-* the License.  You may obtain a copy of the License at
-*
-*      http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.ignite.internal.processors.cache.portable;
-
-import org.apache.ignite.cache.CacheAtomicityMode;
-
-/**
- *
- */
-public class BinaryAtomicCacheLocalEntriesSelfTest extends BinaryTxCacheLocalEntriesSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return CacheAtomicityMode.ATOMIC;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/827b7f61/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/BinaryTxCacheLocalEntriesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/BinaryTxCacheLocalEntriesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/BinaryTxCacheLocalEntriesSelfTest.java
deleted file mode 100644
index e89c73d..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/BinaryTxCacheLocalEntriesSelfTest.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one or more
-* contributor license agreements.  See the NOTICE file distributed with
-* this work for additional information regarding copyright ownership.
-* The ASF licenses this file to You under the Apache License, Version 2.0
-* (the "License"); you may not use this file except in compliance with
-* the License.  You may obtain a copy of the License at
-*
-*      http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.ignite.internal.processors.cache.portable;
-
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.binary.BinaryObject;
-import org.apache.ignite.cache.CachePeekMode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.binary.BinaryMarshaller;
-import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
-
-/**
- *
- */
-public class BinaryTxCacheLocalEntriesSelfTest extends GridCacheAbstractSelfTest {
-    /** */
-    private static final String FIELD = "user-name";
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 2;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception {
-        CacheConfiguration ccfg = super.cacheConfiguration(igniteInstanceName);
-
-        ccfg.setStoreKeepBinary(true);
-
-        return ccfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        cfg.setMarshaller(new BinaryMarshaller());
-
-        return cfg;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLocalEntries() throws Exception {
-        IgniteCache<Integer, BinaryObject> cache = grid(0).cache(DEFAULT_CACHE_NAME).withKeepBinary();
-
-        final int ENTRY_CNT = 10;
-
-        for (int i = 0; i < ENTRY_CNT; i++)
-            cache.put(i, userObject("user-" + i));
-
-        assertEquals(ENTRY_CNT, cache.localSize(CachePeekMode.ALL));
-
-        for (int i = 0; i < gridCount(); i++)
-            jcache(i).withKeepBinary().localEntries();
-
-        cache.removeAll();
-    }
-
-    /**
-     * @param userName User name.
-     * @return Binary object.
-     */
-    private BinaryObject userObject(String userName) {
-        return grid(0).binary().builder("orders").setField(FIELD, userName).build();
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/827b7f61/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java
index c8f0ebe..d9e8594 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java
@@ -51,7 +51,9 @@ import org.apache.ignite.internal.binary.noncompact.BinaryObjectBuilderNonCompac
 import org.apache.ignite.internal.binary.noncompact.BinaryObjectBuilderNonCompactSimpleNameLowerCaseMappersSelfTest;
 import org.apache.ignite.internal.binary.streams.BinaryHeapStreamByteOrderSelfTest;
 import org.apache.ignite.internal.binary.streams.BinaryOffheapStreamByteOrderSelfTest;
+import org.apache.ignite.internal.processors.cache.binary.BinaryAtomicCacheLocalEntriesSelfTest;
 import org.apache.ignite.internal.processors.cache.binary.BinaryMetadataUpdatesFlowTest;
+import org.apache.ignite.internal.processors.cache.binary.BinaryTxCacheLocalEntriesSelfTest;
 import org.apache.ignite.internal.processors.cache.binary.GridCacheBinaryObjectMetadataExchangeMultinodeTest;
 import org.apache.ignite.internal.processors.cache.binary.GridCacheBinaryObjectUserClassloaderSelfTest;
 import org.apache.ignite.internal.processors.cache.binary.GridCacheBinaryStoreBinariesDefaultMappersSelfTest;
@@ -67,8 +69,6 @@ import org.apache.ignite.internal.processors.cache.binary.distributed.replicated
 import org.apache.ignite.internal.processors.cache.binary.local.GridCacheBinaryObjectsAtomicLocalSelfTest;
 import org.apache.ignite.internal.processors.cache.binary.local.GridCacheBinaryObjectsLocalSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteBinaryMetadataUpdateChangingTopologySelfTest;
-import org.apache.ignite.internal.processors.cache.portable.BinaryAtomicCacheLocalEntriesSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.BinaryTxCacheLocalEntriesSelfTest;
 
 /**
  * Test for binary objects stored in cache.

http://git-wip-us.apache.org/repos/asf/ignite/blob/827b7f61/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 9445534..5c8870e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -22,6 +22,7 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -32,7 +33,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
-import java.util.Arrays;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
@@ -404,7 +404,7 @@ public class GridReduceQueryExecutor {
 
             List<ClusterNode> partNodes = assignment.get(partId);
 
-            if (partNodes.size() > 0) {
+            if (!partNodes.isEmpty()) {
                 ClusterNode prim = partNodes.get(0);
 
                 if (!needPartsFilter) {
@@ -501,7 +501,7 @@ public class GridReduceQueryExecutor {
     /**
      * @param schemaName Schema name.
      * @param qry Query.
-     * @param keepPortable Keep portable.
+     * @param keepBinary Keep binary.
      * @param enforceJoinOrder Enforce join order of tables.
      * @param timeoutMillis Timeout in milliseconds.
      * @param cancel Query cancel.
@@ -512,7 +512,7 @@ public class GridReduceQueryExecutor {
     public Iterator<List<?>> query(
         String schemaName,
         GridCacheTwoStepQuery qry,
-        boolean keepPortable,
+        boolean keepBinary,
         boolean enforceJoinOrder,
         int timeoutMillis,
         GridQueryCancel cancel,
@@ -818,7 +818,7 @@ public class GridReduceQueryExecutor {
                     continue;
                 }
 
-                return new GridQueryCacheObjectsIterator(resIter, h2.objectContext(), keepPortable);
+                return new GridQueryCacheObjectsIterator(resIter, h2.objectContext(), keepBinary);
             }
             catch (IgniteCheckedException | RuntimeException e) {
                 U.closeQuiet(r.connection());