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 2017/03/30 11:49:45 UTC

[16/50] [abbrv] ignite git commit: IGNITE-4827: Remove compatibility logic for 1.x versions. This closes #1654.

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/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 347043f..6d8dc14 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
@@ -65,7 +65,6 @@ import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
 import org.apache.ignite.internal.processors.query.GridRunningQueryInfo;
 import org.apache.ignite.internal.processors.query.GridQueryCacheObjectsIterator;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
-import org.apache.ignite.internal.processors.query.GridRunningQueryInfo;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryContext;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlSortColumn;
@@ -74,7 +73,6 @@ import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQuery
 import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponse;
 import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequest;
 import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageResponse;
-import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryRequest;
 import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.typedef.CIX2;
@@ -113,9 +111,6 @@ import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuerySpl
  */
 public class GridReduceQueryExecutor {
     /** */
-    private static final IgniteProductVersion DISTRIBUTED_JOIN_SINCE = IgniteProductVersion.fromString("1.7.0");
-
-    /** */
     private static final String MERGE_INDEX_UNSORTED = "merge_scan";
 
     /** */
@@ -623,9 +618,6 @@ public class GridReduceQueryExecutor {
                             .parameters(mapQry.parameters(), mapQry.parameterIndexes()));
                 }
 
-                IgniteProductVersion minNodeVer = cctx.shared().exchange().minimumNodeVersion(topVer);
-
-                final boolean oldStyle = minNodeVer.compareToIgnoreTimestamp(DISTRIBUTED_JOIN_SINCE) < 0;
                 final boolean distributedJoins = qry.distributedJoins();
 
                 final Collection<ClusterNode> finalNodes = nodes;
@@ -638,9 +630,6 @@ public class GridReduceQueryExecutor {
 
                 boolean retry = false;
 
-                if (oldStyle && distributedJoins)
-                    throw new CacheException("Failed to enable distributed joins. Topology contains older data nodes.");
-
                 // Always enforce join order on map side to have consistent behavior.
                 int flags = GridH2QueryRequest.FLAG_ENFORCE_JOIN_ORDER;
 
@@ -654,15 +643,6 @@ public class GridReduceQueryExecutor {
                     flags |= GridH2QueryRequest.FLAG_EXPLAIN;
 
                 if (send(nodes,
-                    oldStyle ?
-                        new GridQueryRequest(qryReqId,
-                            r.pageSize,
-                            space,
-                            mapQrys,
-                            topVer,
-                            extraSpaces(space, qry.spaces()),
-                            null,
-                            timeoutMillis) :
                         new GridH2QueryRequest()
                             .requestId(qryReqId)
                             .topologyVersion(topVer)
@@ -673,7 +653,7 @@ public class GridReduceQueryExecutor {
                             .queries(mapQrys)
                             .flags(flags)
                             .timeout(timeoutMillis),
-                    oldStyle && partsMap != null ? new ExplicitPartitionsSpecializer(partsMap) : null,
+                    null,
                     false)) {
 
                     awaitAllReplies(r, nodes, cancel);
@@ -1215,26 +1195,6 @@ public class GridReduceQueryExecutor {
     }
 
     /**
-     * @param msg Message to copy.
-     * @param node Node.
-     * @param partsMap Partitions map.
-     * @return Copy of message with partitions set.
-     */
-    private Message copy(Message msg, ClusterNode node, Map<ClusterNode,IntArray> partsMap) {
-        assert partsMap != null;
-
-        GridQueryRequest res = new GridQueryRequest((GridQueryRequest)msg);
-
-        IntArray parts = partsMap.get(node);
-
-        assert parts != null : node;
-
-        res.partitions(toArray(parts));
-
-        return res;
-    }
-
-    /**
      * @param ints Ints.
      * @return Array.
      */
@@ -1458,24 +1418,4 @@ public class GridReduceQueryExecutor {
             state(e, null);
         }
     }
-
-    /**
-     *
-     */
-    private class ExplicitPartitionsSpecializer implements IgniteBiClosure<ClusterNode,Message,Message> {
-        /** */
-        private final Map<ClusterNode,IntArray> partsMap;
-
-        /**
-         * @param partsMap Partitions map.
-         */
-        private ExplicitPartitionsSpecializer(Map<ClusterNode,IntArray> partsMap) {
-            this.partsMap = partsMap;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Message apply(ClusterNode n, Message msg) {
-            return copy(msg, n, partsMap);
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
index 04a68ae..19b04f2 100755
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
@@ -431,15 +431,12 @@ class VisorCacheCommand {
 
                             formatDouble(nm.getCurrentCpuLoad * 100d) + " %",
                             X.timeSpan2HMSM(nm.getUpTime),
-                            cm match {
-                                case v2: VisorCacheMetricsV2 => (
-                                    "Total: " + (v2.keySize() + v2.offHeapEntriesCount()),
-                                    "  Heap: " + v2.keySize(),
-                                    "  Off-Heap: " + v2.offHeapEntriesCount(),
-                                    "  Off-Heap Memory: " + formatMemory(v2.offHeapAllocatedSize())
-                                )
-                                case v1 => v1.keySize()
-                            },
+                            (
+                                "Total: " + (cm.keySize() + cm.offHeapEntriesCount()),
+                                "  Heap: " + cm.keySize(),
+                                "  Off-Heap: " + cm.offHeapEntriesCount(),
+                                "  Off-Heap Memory: " + formatMemory(cm.offHeapAllocatedSize())
+                            ),
                             (
                                 "Hi: " + cm.hits(),
                                 "Mi: " + cm.misses(),
@@ -923,7 +920,7 @@ object VisorCacheCommand {
         cacheT += ("Store Class", safe(storeCfg.store()))
         cacheT += ("Store Factory Class", storeCfg.storeFactory())
         cacheT += ("Store Keep Binary", storeCfg match {
-            case cfg: VisorCacheStoreConfigurationV2 => cfg.storeKeepBinary()
+            case cfg: VisorCacheStoreConfiguration => cfg.storeKeepBinary()
             case _ => false
         })
         cacheT += ("Store Read Through", bool2Str(storeCfg.readThrough()))
@@ -949,7 +946,7 @@ object VisorCacheCommand {
 
         cacheT +=("Query Execution Time Threshold", queryCfg.longQueryWarningTimeout())
         cacheT +=("Query Schema Name", queryCfg match {
-            case cfg: VisorCacheQueryConfigurationV2 => cfg.sqlSchema()
+            case cfg: VisorCacheQueryConfiguration => cfg.sqlSchema()
             case _ => null
         })
         cacheT +=("Query Escaped Names", bool2Str(queryCfg.sqlEscapeAll()))

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala
index 3aa2a19..9668072 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala
@@ -139,7 +139,7 @@ class VisorCacheScanCommand {
         val firstPage =
             try
                 executeRandom(groupForDataNode(node, cacheName),
-                    classOf[VisorQueryTask], new VisorQueryArg(cacheName, null, false, pageSize)) match {
+                    classOf[VisorQueryTask], new VisorQueryArg(cacheName, null, false, false, false, pageSize)) match {
                     case x if x.get1() != null =>
                         error(x.get1())
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/disco/VisorDiscoveryCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/disco/VisorDiscoveryCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/disco/VisorDiscoveryCommand.scala
index 9ff7118..b0f2c82 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/disco/VisorDiscoveryCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/disco/VisorDiscoveryCommand.scala
@@ -216,7 +216,7 @@ class VisorDiscoveryCommand extends VisorConsoleCommand {
 
         if (nodeStartTime > System.currentTimeMillis() - tmFrame) {
             val root = new VisorGridDiscoveryEvent(EVT_NODE_JOINED, null, U.gridEventName(EVT_NODE_JOINED),
-                node.id(), nodeStartTime, "", "", node.id, node.addresses().head, node.isDaemon)
+                node.id(), nodeStartTime, "", "", node.id, node.addresses().head, node.isDaemon, 0)
 
             evts = Seq(root) ++ evts
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/WaitMapExchangeFinishCallable.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/WaitMapExchangeFinishCallable.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/WaitMapExchangeFinishCallable.java
index ac91b51..83c50bd 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/WaitMapExchangeFinishCallable.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/WaitMapExchangeFinishCallable.java
@@ -25,7 +25,7 @@ import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap2;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.yardstickframework.BenchmarkUtils;
@@ -53,7 +53,7 @@ public class WaitMapExchangeFinishCallable implements IgniteCallable<Void> {
                     boolean success = true;
 
                     if (top.topologyVersion().topologyVersion() == ignite.cluster().topologyVersion()) {
-                        for (Map.Entry<UUID, GridDhtPartitionMap2> e : top.partitionMap(true).entrySet()) {
+                        for (Map.Entry<UUID, GridDhtPartitionMap> e : top.partitionMap(true).entrySet()) {
                             for (Map.Entry<Integer, GridDhtPartitionState> p : e.getValue().entrySet()) {
                                 if (p.getValue() != GridDhtPartitionState.OWNING) {
                                     BenchmarkUtils.println("Not owning partition [part=" + p.getKey() +

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteFailoverAbstractBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteFailoverAbstractBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteFailoverAbstractBenchmark.java
index a025bb5..b90573b 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteFailoverAbstractBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteFailoverAbstractBenchmark.java
@@ -38,7 +38,7 @@ import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap2;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteRunnable;
 import org.apache.ignite.mxbean.IgniteMXBean;
@@ -188,7 +188,7 @@ public abstract class IgniteFailoverAbstractBenchmark<K, V> extends IgniteCacheA
 
                 GridDhtPartitionFullMap partMap = dht.topology().partitionMap(true);
 
-                for (Map.Entry<UUID, GridDhtPartitionMap2> e : partMap.entrySet()) {
+                for (Map.Entry<UUID, GridDhtPartitionMap> e : partMap.entrySet()) {
                     log.info("Checking node: " + e.getKey());
 
                     for (Map.Entry<Integer, GridDhtPartitionState> e1 : e.getValue().entrySet()) {