http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityUtils.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityUtils.java index c689be0..f4ba945 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityUtils.java @@ -158,7 +158,7 @@ class GridAffinityUtils { assert ignite != null; assert log != null; - GridKernal kernal = ((GridKernal) ignite); + IgniteKernal kernal = ((IgniteKernal) ignite); GridCacheContext<Object, Object> cctx = kernal.internalCache(cacheName).context();
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java index 92d2039..35b34f0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java @@ -5249,7 +5249,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, /** {@inheritDoc} */ @Override public Object call() throws Exception { - ((GridEx) ignite).cachex(cacheName).clearAll(); + ((IgniteEx) ignite).cachex(cacheName).clearAll(); return null; } @@ -5302,7 +5302,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, /** {@inheritDoc} */ @Override public Integer apply(Object o) { - GridCache<Object, Object> cache = ((GridEx)ignite).cachex(cacheName); + GridCache<Object, Object> cache = ((IgniteEx)ignite).cachex(cacheName); return primaryOnly ? cache.primarySize() : cache.size(); } @@ -5533,7 +5533,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>, /** {@inheritDoc} */ @Override public Void call() throws Exception { - GridCacheAdapter<K, V> cache = ((GridKernal)ignite).context().cache().internalCache(cacheName); + GridCacheAdapter<K, V> cache = ((IgniteKernal)ignite).context().cache().internalCache(cacheName); assert cache != null : cacheName; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/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 0336832..0f5b06d 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 @@ -530,7 +530,7 @@ public class GridCacheContext<K, V> implements Externalizable { /** * @return Grid instance. */ - public GridEx grid() { + public IgniteEx grid() { return ctx.grid(); } @@ -1888,7 +1888,7 @@ public class GridCacheContext<K, V> implements Externalizable { try { IgniteBiTuple<String, String> t = stash.get(); - GridKernal grid = GridGainEx.gridx(t.get1()); + IgniteKernal grid = GridGainEx.gridx(t.get1()); GridCacheAdapter<K, V> cache = grid.internalCache(t.get2()); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresManager.java index d49ea9f..2793bb2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresManager.java @@ -1493,7 +1493,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager /** {@inheritDoc} */ @Override public Void call() throws IgniteCheckedException { - GridCacheAdapter cache = ((GridKernal) ignite).context().cache().internalCache(cacheName); + GridCacheAdapter cache = ((IgniteKernal) ignite).context().cache().internalCache(cacheName); assert cache != null; @@ -1561,7 +1561,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager /** {@inheritDoc} */ @Override public Void call() throws IgniteCheckedException { - GridCacheAdapter cache = ((GridKernal) ignite).context().cache().internalCache(cacheName); + GridCacheAdapter cache = ((IgniteKernal) ignite).context().cache().internalCache(cacheName); assert cache != null; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java index d113cda..4ba6149 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java @@ -1882,7 +1882,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte /** {@inheritDoc} */ @Override public Collection<CacheSqlMetadata> call() { - final GridKernalContext ctx = ((GridKernal) ignite).context(); + final GridKernalContext ctx = ((IgniteKernal) ignite).context(); Collection<String> cacheNames = F.viewReadOnly(ctx.cache().caches(), new C1<GridCache<?, ?>, String>() { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java index 64fc88e..b3f99a3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java @@ -104,7 +104,7 @@ public class GridCacheQueryJdbcMetadataTask extends ComputeTaskAdapter<String, b byte[] data; try { - GridCache<?, ?> cache = ((GridEx) ignite).cachex(cacheName); + GridCache<?, ?> cache = ((IgniteEx) ignite).cachex(cacheName); assert cache != null; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java index 5371504..acd5051 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java @@ -192,7 +192,7 @@ public class GridCacheQueryJdbcTask extends ComputeTaskAdapter<byte[], byte[]> { assert args != null; assert futId == null; - GridCache<?, ?> cache = ((GridEx) ignite).cachex(cacheName); + GridCache<?, ?> cache = ((IgniteEx) ignite).cachex(cacheName); CacheQuery<List<?>> qry = ((GridCacheQueriesEx<?, ?>)cache.queries()).createSqlFieldsQuery(sql, true); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/dr/GridDrDataLoadCacheUpdater.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/dr/GridDrDataLoadCacheUpdater.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/dr/GridDrDataLoadCacheUpdater.java index 03216a2..6e7e385 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/dr/GridDrDataLoadCacheUpdater.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/dr/GridDrDataLoadCacheUpdater.java @@ -40,7 +40,7 @@ public class GridDrDataLoadCacheUpdater<K, V> implements IgniteDataLoadCacheUpda throws IgniteCheckedException { String cacheName = cache0.getConfiguration(CacheConfiguration.class).getName(); - GridKernalContext ctx = ((GridKernal)cache0.unwrap(Ignite.class)).context(); + GridKernalContext ctx = ((IgniteKernal)cache0.unwrap(Ignite.class)).context(); IgniteLogger log = ctx.log(GridDrDataLoadCacheUpdater.class); GridCacheAdapter<K, V> cache = ctx.cache().internalCache(cacheName); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsImpl.java index 2e20e62..01c3edc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsImpl.java @@ -2013,7 +2013,7 @@ public final class GridGgfsImpl implements GridGgfsEx { private Ignite g; @Nullable @Override public IgniteBiTuple<Long, Long> execute() throws IgniteCheckedException { - IgniteFs ggfs = ((GridKernal)g).context().ggfs().ggfs(ggfsName); + IgniteFs ggfs = ((IgniteKernal)g).context().ggfs().ggfs(ggfsName); if (ggfs == null) return F.t(0L, 0L); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java index fb55ab9..063509b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java @@ -60,7 +60,7 @@ public class GridResourceProcessor extends GridProcessorAdapter { IgniteServiceResource.class); /** Grid instance injector. */ - private GridResourceBasicInjector<GridEx> gridInjector; + private GridResourceBasicInjector<IgniteEx> gridInjector; /** Spring application context injector. */ private GridResourceInjector springCtxInjector; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java index 9497936..e9c0d99 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java @@ -433,7 +433,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter { final Long d = delta; - return ((GridKernal)cache.gridProjection().ignite()).context().closure().callLocalSafe(new Callable<Object>() { + return ((IgniteKernal)cache.gridProjection().ignite()).context().closure().callLocalSafe(new Callable<Object>() { @Override public Object call() throws Exception { return l.addAndGet(decr ? -d : d); } @@ -719,7 +719,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter { // Need to apply both operation and response transformation remotely // as cache could be inaccessible on local node and // exception processing should be consistent with local execution. - return op.apply((CacheProjection<Object, Object>)prj, ((GridKernal)g).context()). + return op.apply((CacheProjection<Object, Object>)prj, ((IgniteKernal)g).context()). chain(resultWrapper((CacheProjection<Object, Object>)prj, key)).get(); } } @@ -768,7 +768,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter { // Need to apply both operation and response transformation remotely // as cache could be inaccessible on local node and // exception processing should be consistent with local execution. - return op.apply(cache, ((GridKernal)g).context()).chain(resultWrapper(cache, key)).get(); + return op.apply(cache, ((IgniteKernal)g).context()).chain(resultWrapper(cache, key)).get(); } } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java index 5178847..87d5dba 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java @@ -590,7 +590,7 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter { /** */ @IgniteInstanceResource - private GridEx g; + private IgniteEx g; /** * Required by {@link Externalizable}. http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java index 86818d5..1515a8a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java @@ -341,7 +341,7 @@ class GridServiceProxy<T> implements Serializable { /** {@inheritDoc} */ @Override public Object call() throws Exception { - ManagedServiceContextImpl svcCtx = ((GridKernal) ignite).context().service().serviceContext(svcName); + ManagedServiceContextImpl svcCtx = ((IgniteKernal) ignite).context().service().serviceContext(svcName); if (svcCtx == null) throw new GridServiceNotFoundException(svcName); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorJob.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorJob.java index 35f11d9..0950a4e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorJob.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorJob.java @@ -31,7 +31,7 @@ import static org.apache.ignite.internal.visor.util.VisorTaskUtils.*; */ public abstract class VisorJob<A, R> extends ComputeJobAdapter { @IgniteInstanceResource - protected transient GridEx g; + protected transient IgniteEx g; /** Job start time. */ protected transient long start; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java index 081213f..16ead12 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java @@ -37,7 +37,7 @@ import static org.apache.ignite.internal.visor.util.VisorTaskUtils.*; */ public abstract class VisorMultiNodeTask<A, R, J> implements ComputeTask<VisorTaskArgument<A>, R> { @IgniteInstanceResource - protected GridEx g; + protected IgniteEx g; /** Debug flag. */ protected boolean debug; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java index f49ab87..6967969 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java @@ -103,7 +103,7 @@ public class VisorCache implements Serializable { String cacheName = c.name(); - GridCacheAdapter ca = ((GridKernal)g).internalCache(cacheName); + GridCacheAdapter ca = ((IgniteKernal)g).internalCache(cacheName); long swapSize; long swapKeys; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorComputeMonitoringHolder.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorComputeMonitoringHolder.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorComputeMonitoringHolder.java index 2cabdcc..1a2dd79 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorComputeMonitoringHolder.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorComputeMonitoringHolder.java @@ -47,7 +47,7 @@ public class VisorComputeMonitoringHolder { * @param g grid. * @param visorKey unique Visor instance key. */ - public void startCollect(GridEx g, String visorKey) { + public void startCollect(IgniteEx g, String visorKey) { synchronized(listenVisor) { if (cleanupStopped) { scheduleCleanupJob(g); @@ -66,7 +66,7 @@ public class VisorComputeMonitoringHolder { * @param g grid. * @return {@code true} if task events should remain enabled. */ - private boolean tryDisableEvents(GridEx g) { + private boolean tryDisableEvents(IgniteEx g) { if (!listenVisor.values().contains(true)) { listenVisor.clear(); @@ -82,7 +82,7 @@ public class VisorComputeMonitoringHolder { * @param g grid. * @param visorKey uniq Visor instance key. */ - public void stopCollect(GridEx g, String visorKey) { + public void stopCollect(IgniteEx g, String visorKey) { synchronized(listenVisor) { listenVisor.remove(visorKey); @@ -94,8 +94,8 @@ public class VisorComputeMonitoringHolder { * Schedule cleanup process for events monitoring. * @param g grid. */ - private void scheduleCleanupJob(final GridEx g) { - ((GridKernal)g).context().timeout().addTimeoutObject(new GridTimeoutObjectAdapter(CLEANUP_TIMEOUT) { + private void scheduleCleanupJob(final IgniteEx g) { + ((IgniteKernal)g).context().timeout().addTimeoutObject(new GridTimeoutObjectAdapter(CLEANUP_TIMEOUT) { @Override public void onTimeout() { synchronized(listenVisor) { if (tryDisableEvents(g)) { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorBasicConfiguration.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorBasicConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorBasicConfiguration.java index ae68e38..6498e45 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorBasicConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorBasicConfiguration.java @@ -107,7 +107,7 @@ public class VisorBasicConfiguration implements Serializable { * @param c Grid configuration. * @return Data transfer object for node basic configuration properties. */ - public static VisorBasicConfiguration from(GridEx g, IgniteConfiguration c) { + public static VisorBasicConfiguration from(IgniteEx g, IgniteConfiguration c) { VisorBasicConfiguration cfg = new VisorBasicConfiguration(); cfg.gridName(c.getGridName()); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java index f9eb9ca..66db0c6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java @@ -90,7 +90,7 @@ public class VisorGridConfiguration implements Serializable { * @param g Grid. * @return Fill data transfer object with node configuration data. */ - public VisorGridConfiguration from(GridEx g) { + public VisorGridConfiguration from(IgniteEx g) { assert g != null; IgniteConfiguration c = g.configuration(); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java index f00f47b..753e99e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java @@ -123,7 +123,7 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa /** Collect GGFS. */ private void ggfs(VisorNodeDataCollectorJobResult res) { try { - IgniteFsProcessorAdapter ggfsProc = ((GridKernal)g).context().ggfs(); + IgniteFsProcessorAdapter ggfsProc = ((IgniteKernal)g).context().ggfs(); for (IgniteFs ggfs : ggfsProc.ggfss()) { long start0 = U.currentTimeMillis(); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java index 3925dd0..7b60a0e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java @@ -281,7 +281,7 @@ public class VisorQueryTask extends VisorOneNodeTask<VisorQueryTask.VisorQueryAr * @param id Uniq query result id. */ private void scheduleResultSetHolderRemoval(final String id) { - ((GridKernal)g).context().timeout() + ((IgniteKernal)g).context().timeout() .addTimeoutObject(new GridTimeoutObjectAdapter(VisorQueryUtils.RMV_DELAY) { @Override public void onTimeout() { ClusterNodeLocalMap<String, VisorFutureResultSetHolder> storage = g.nodeLocalMap(); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/streamer/router/StreamerCacheAffinityEventRouter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/streamer/router/StreamerCacheAffinityEventRouter.java b/modules/core/src/main/java/org/apache/ignite/streamer/router/StreamerCacheAffinityEventRouter.java index e0bd722..8cb3996 100644 --- a/modules/core/src/main/java/org/apache/ignite/streamer/router/StreamerCacheAffinityEventRouter.java +++ b/modules/core/src/main/java/org/apache/ignite/streamer/router/StreamerCacheAffinityEventRouter.java @@ -58,7 +58,7 @@ public class StreamerCacheAffinityEventRouter extends StreamerEventRouterAdapter if (evt instanceof CacheAffinityEvent) { CacheAffinityEvent e = (CacheAffinityEvent)evt; - GridCache<Object, Object> c = ((GridEx) ignite).cachex(e.cacheName()); + GridCache<Object, Object> c = ((IgniteEx) ignite).cachex(e.cacheName()); assert c != null; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/GridExceptionHelpLinksSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/GridExceptionHelpLinksSelfTest.java b/modules/core/src/test/java/org/apache/ignite/GridExceptionHelpLinksSelfTest.java deleted file mode 100644 index f4bcb23..0000000 --- a/modules/core/src/test/java/org/apache/ignite/GridExceptionHelpLinksSelfTest.java +++ /dev/null @@ -1,106 +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; - -import junit.framework.*; -import org.apache.ignite.*; -import org.jetbrains.annotations.*; - -import java.util.*; - -import static org.apache.ignite.internal.util.GridUtils.*; - -/** - * Tests for proper link output in stack traces. - */ -public class GridExceptionHelpLinksSelfTest extends TestCase { - /** - * @throws Exception If failed. - */ - public void testDefaultLinks() throws Exception { - assertTrue(hasLinksInMessage(new IgniteCheckedException("test"), DFLT_HELP_LINKS)); - assertTrue(hasLinksInMessage(new IgniteCheckedException(new Exception()), DFLT_HELP_LINKS)); - assertTrue(hasLinksInMessage(new IgniteCheckedException("test", new Exception()), DFLT_HELP_LINKS)); - - assertTrue(hasLinksInMessage(new IgniteException("test"), DFLT_HELP_LINKS)); - assertTrue(hasLinksInMessage(new IgniteException(new Exception()), DFLT_HELP_LINKS)); - assertTrue(hasLinksInMessage(new IgniteException("test", new Exception()), DFLT_HELP_LINKS)); - } - - /** - * Tests default links suppression. - */ - public void testLinksUniqueness() { - assertLinksAppearOnce( - new IgniteCheckedException("test", - new IgniteCheckedException("test nested", - new IgniteCheckedException("last"))), - DFLT_HELP_LINKS); - - assertLinksAppearOnce( - new IgniteException("test", - new IgniteException("test nested", - new IgniteException("last"))), - DFLT_HELP_LINKS); - - assertLinksAppearOnce( - new IgniteCheckedException("test", - new IgniteException("test nested", - new IgniteCheckedException("last"))), - DFLT_HELP_LINKS); - - assertLinksAppearOnce( - new IgniteException("test", - new IgniteCheckedException("test nested", - new IgniteException("last"))), - DFLT_HELP_LINKS); - } - - /** - * @param e Root exception. - * @param links Set of links to ensure present only once in full stack trace. - */ - @SuppressWarnings("TypeMayBeWeakened") - private void assertLinksAppearOnce(Throwable e, List<String>... links) { - Set<List<String>> seen = new HashSet<>(); - - while (e != null) { - for (List<String> l : links) - if (hasLinksInMessage(e, l)) - assertTrue(seen.add(l)); - - e = e.getCause(); - } - } - - /** - * @param e Exception - * @param links List of links. - * @return Whether exception has all passed links in it's message. - */ - private boolean hasLinksInMessage(Throwable e, @Nullable Iterable<String> links) { - if (links == null) - return true; - - for (String link : links) - if (!e.getMessage().contains(link)) - return false; - - return true; - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/GridExternalizableAbstractTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/GridExternalizableAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/GridExternalizableAbstractTest.java deleted file mode 100644 index e4cdae9..0000000 --- a/modules/core/src/test/java/org/apache/ignite/GridExternalizableAbstractTest.java +++ /dev/null @@ -1,41 +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; - -import org.apache.ignite.marshaller.*; -import org.apache.ignite.marshaller.jdk.*; -import org.apache.ignite.marshaller.optimized.*; -import org.apache.ignite.testframework.junits.common.*; -import java.util.*; - -/** - * Base externalizable test class. - */ -public class GridExternalizableAbstractTest extends GridCommonAbstractTest { - /** - * @return Marshallers. - */ - protected List<IgniteMarshaller> getMarshallers() { - List<IgniteMarshaller> marshallers = new ArrayList<>(); - - marshallers.add(new IgniteJdkMarshaller()); - marshallers.add(new IgniteOptimizedMarshaller()); - - return marshallers; - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/IgniteExceptionHelpLinksSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/IgniteExceptionHelpLinksSelfTest.java b/modules/core/src/test/java/org/apache/ignite/IgniteExceptionHelpLinksSelfTest.java new file mode 100644 index 0000000..3fbf4ec --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/IgniteExceptionHelpLinksSelfTest.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite; + +import junit.framework.*; +import org.apache.ignite.*; +import org.jetbrains.annotations.*; + +import java.util.*; + +import static org.apache.ignite.internal.util.GridUtils.*; + +/** + * Tests for proper link output in stack traces. + */ +public class IgniteExceptionHelpLinksSelfTest extends TestCase { + /** + * @throws Exception If failed. + */ + public void testDefaultLinks() throws Exception { + assertTrue(hasLinksInMessage(new IgniteCheckedException("test"), DFLT_HELP_LINKS)); + assertTrue(hasLinksInMessage(new IgniteCheckedException(new Exception()), DFLT_HELP_LINKS)); + assertTrue(hasLinksInMessage(new IgniteCheckedException("test", new Exception()), DFLT_HELP_LINKS)); + + assertTrue(hasLinksInMessage(new IgniteException("test"), DFLT_HELP_LINKS)); + assertTrue(hasLinksInMessage(new IgniteException(new Exception()), DFLT_HELP_LINKS)); + assertTrue(hasLinksInMessage(new IgniteException("test", new Exception()), DFLT_HELP_LINKS)); + } + + /** + * Tests default links suppression. + */ + public void testLinksUniqueness() { + assertLinksAppearOnce( + new IgniteCheckedException("test", + new IgniteCheckedException("test nested", + new IgniteCheckedException("last"))), + DFLT_HELP_LINKS); + + assertLinksAppearOnce( + new IgniteException("test", + new IgniteException("test nested", + new IgniteException("last"))), + DFLT_HELP_LINKS); + + assertLinksAppearOnce( + new IgniteCheckedException("test", + new IgniteException("test nested", + new IgniteCheckedException("last"))), + DFLT_HELP_LINKS); + + assertLinksAppearOnce( + new IgniteException("test", + new IgniteCheckedException("test nested", + new IgniteException("last"))), + DFLT_HELP_LINKS); + } + + /** + * @param e Root exception. + * @param links Set of links to ensure present only once in full stack trace. + */ + @SuppressWarnings("TypeMayBeWeakened") + private void assertLinksAppearOnce(Throwable e, List<String>... links) { + Set<List<String>> seen = new HashSet<>(); + + while (e != null) { + for (List<String> l : links) + if (hasLinksInMessage(e, l)) + assertTrue(seen.add(l)); + + e = e.getCause(); + } + } + + /** + * @param e Exception + * @param links List of links. + * @return Whether exception has all passed links in it's message. + */ + private boolean hasLinksInMessage(Throwable e, @Nullable Iterable<String> links) { + if (links == null) + return true; + + for (String link : links) + if (!e.getMessage().contains(link)) + return false; + + return true; + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/IgniteExternalizableAbstractTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/IgniteExternalizableAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/IgniteExternalizableAbstractTest.java new file mode 100644 index 0000000..1bae7cd --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/IgniteExternalizableAbstractTest.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite; + +import org.apache.ignite.marshaller.*; +import org.apache.ignite.marshaller.jdk.*; +import org.apache.ignite.marshaller.optimized.*; +import org.apache.ignite.testframework.junits.common.*; +import java.util.*; + +/** + * Base externalizable test class. + */ +public class IgniteExternalizableAbstractTest extends GridCommonAbstractTest { + /** + * @return Marshallers. + */ + protected List<IgniteMarshaller> getMarshallers() { + List<IgniteMarshaller> marshallers = new ArrayList<>(); + + marshallers.add(new IgniteJdkMarshaller()); + marshallers.add(new IgniteOptimizedMarshaller()); + + return marshallers; + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinityNodesSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinityNodesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinityNodesSelfTest.java index 92a8598..28e5325 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinityNodesSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinityNodesSelfTest.java @@ -192,13 +192,13 @@ public class GridCachePartitionFairAffinityNodesSelfTest extends GridCommonAbstr info("Grid 0: " + grid(0).localNode().id()); - ((GridKernal)grid(0)).internalCache().context().affinity().affinityReadyFuture(topVer).get(); + ((IgniteKernal)grid(0)).internalCache().context().affinity().affinityReadyFuture(topVer).get(); for (int i : started) { if (i != 0) { - GridEx grid = grid(i); + IgniteEx grid = grid(i); - ((GridKernal)grid).internalCache().context().affinity().affinityReadyFuture(topVer).get(); + ((IgniteKernal)grid).internalCache().context().affinity().affinityReadyFuture(topVer).get(); info("Grid " + i + ": " + grid.localNode().id()); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java index 09c4e4f..7b080af 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java @@ -232,7 +232,7 @@ public class GridCacheJdbcBlobStoreMultithreadedSelfTest extends GridCommonAbstr assertEquals(GRID_CNT, Ignition.allGrids().size()); for (Ignite ignite : Ignition.allGrids()) { - GridCacheContext cctx = ((GridKernal)ignite).internalCache().context(); + GridCacheContext cctx = ((IgniteKernal)ignite).internalCache().context(); CacheStore store = cctx.store().configuredStore(); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/fs/GridGgfsFragmentizerSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/fs/GridGgfsFragmentizerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/fs/GridGgfsFragmentizerSelfTest.java index dacb3c8..0756be1 100644 --- a/modules/core/src/test/java/org/apache/ignite/fs/GridGgfsFragmentizerSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/fs/GridGgfsFragmentizerSelfTest.java @@ -243,7 +243,7 @@ public class GridGgfsFragmentizerSelfTest extends GridGgfsFragmentizerAbstractSe GridTestUtils.retryAssert(log, 50, 100, new CA() { @Override public void apply() { for (int i = 0; i < NODE_CNT; i++) { - GridEx g = grid(i); + IgniteEx g = grid(i); GridCache<Object, Object> cache = g.cachex(DATA_CACHE_NAME); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoveryEventSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoveryEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoveryEventSelfTest.java index d57c8ee..106cd2a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoveryEventSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoveryEventSelfTest.java @@ -21,7 +21,6 @@ import org.apache.ignite.*; import org.apache.ignite.cluster.*; import org.apache.ignite.configuration.*; import org.apache.ignite.events.*; -import org.apache.ignite.internal.*; import org.apache.ignite.lang.*; import org.apache.ignite.spi.discovery.tcp.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; @@ -403,7 +402,7 @@ public class GridDiscoveryEventSelfTest extends GridCommonAbstractTest { daemon = true; - GridKernal daemon = (GridKernal)startGrid(3); + IgniteKernal daemon = (IgniteKernal)startGrid(3); IgniteDiscoveryEvent join = daemon.context().discovery().localJoinEvent(); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java index 38575c3..15cfa6c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java @@ -127,7 +127,7 @@ public class GridDiscoverySelfTest extends GridCommonAbstractTest { nodes.add(new GridDiscoveryTestNode()); @SuppressWarnings("deprecation") - long hash = ((GridKernal) ignite).context().discovery().topologyHash(nodes); + long hash = ((IgniteKernal) ignite).context().discovery().topologyHash(nodes); boolean isHashed = hashes.add(hash); @@ -236,7 +236,7 @@ public class GridDiscoverySelfTest extends GridCommonAbstractTest { */ public void testCacheNodes() throws Exception { // Validate only original node is available. - GridDiscoveryManager discoMgr = ((GridKernal) ignite).context().discovery(); + GridDiscoveryManager discoMgr = ((IgniteKernal) ignite).context().discovery(); Collection<ClusterNode> nodes = discoMgr.allNodes(); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/GridExecutorServiceTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridExecutorServiceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridExecutorServiceTest.java deleted file mode 100644 index ac20e67..0000000 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridExecutorServiceTest.java +++ /dev/null @@ -1,315 +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; - -import org.apache.ignite.*; -import org.apache.ignite.internal.*; -import org.apache.ignite.resources.*; -import org.apache.ignite.internal.executor.*; -import org.apache.ignite.internal.util.typedef.*; -import org.apache.ignite.testframework.junits.common.*; - -import java.io.*; -import java.util.*; -import java.util.concurrent.*; - -/** - * Grid distributed executor test. - */ -@GridCommonTest(group = "Thread Tests") -public class GridExecutorServiceTest extends GridCommonAbstractTest { - /** */ - public GridExecutorServiceTest() { - super(true); - } - - /** - * @throws Exception Thrown in case of test failure. - */ - public void testExecute() throws Exception { - Ignite ignite = G.ignite(getTestGridName()); - - ExecutorService srvc = createExecutorService(ignite); - - srvc.execute(new Runnable() { - @IgniteInstanceResource - private Ignite ignite; - - @Override public void run() { - System.out.println("Test message."); - - assert this.ignite != null; - } - }); - - srvc.execute(new TestRunnable()); - - srvc.shutdown(); - } - - /** - * @throws Exception Thrown in case of test failure. - */ - public void testSubmit() throws Exception { - Ignite ignite = G.ignite(getTestGridName()); - - ExecutorService srvc = createExecutorService(ignite); - - Future<?> fut = srvc.submit(new TestRunnable()); - - Object res = fut.get(); - - info("Default Runnable result:" + res); - - assert res == null : "Failed to get valid default result for submitted Runnable: " + res; - - String val = "test-value"; - - fut = srvc.submit(new TestRunnable(), val); - - res = fut.get(); - - info("Defined Runnable result:" + res); - - assert val.equals(res) : "Failed to get valid predefined result for submitted Runnable: " + res; - - fut = srvc.submit(new TestCallable<>(val)); - - res = fut.get(); - - info("Callable result:" + res); - - assert val.equals(res) : "Failed to get valid result for submitted Callable: " + res; - - srvc.shutdown(); - } - - /** - * @throws Exception Thrown in case of test failure. - */ - public void testSubmitWithFutureTimeout() throws Exception { - Ignite ignite = G.ignite(getTestGridName()); - - ExecutorService srvc = createExecutorService(ignite); - - Future<Integer> fut = srvc.submit(new TestCallable<>(3000)); // Just sleep for 3 seconds. - - boolean ok = true; - - try { - fut.get(1, TimeUnit.SECONDS); - - ok = false; - } - catch (TimeoutException e) { - info("Task timeout elapsed: " + e.getMessage()); - } - - assert ok : "Timeout must be thrown."; - - srvc.shutdown(); - } - - /** - * @throws Exception Thrown in case of test failure. - */ - @SuppressWarnings("TooBroadScope") - public void testInvokeAll() throws Exception { - Ignite ignite = G.ignite(getTestGridName()); - - ExecutorService srvc = createExecutorService(ignite); - - Collection<Callable<String>> cmds = new ArrayList<>(2); - - String val1 = "test-value-1"; - String val2 = "test-value-2"; - - cmds.add(new TestCallable<>(val1)); - cmds.add(new TestCallable<>(val2)); - - List<Future<String>> futs = srvc.invokeAll(cmds); - - assert futs != null; - assert futs.size() == 2; - - String res1 = futs.get(0).get(); - String res2 = futs.get(1).get(); - - assert val1.equals(res1) : "Failed to get valid result for first command: " + res1; - assert val2.equals(res2) : "Failed to get valid result for second command: " + res2; - - srvc.shutdown(); - } - - /** - * @throws Exception Thrown in case of test failure. - */ - @SuppressWarnings("TooBroadScope") - public void testInvokeAllWithTimeout() throws Exception { - Ignite ignite = G.ignite(getTestGridName()); - - ExecutorService srvc = createExecutorService(ignite); - - Collection<Callable<Integer>> cmds = new ArrayList<>(); - - cmds.add(new TestCallable<>(3000)); // Just sleeps for 3 seconds. - cmds.add(new TestCallable<>(3000)); // Just sleeps for 3 seconds. - - List<Future<Integer>> fut = srvc.invokeAll(cmds, 1, TimeUnit.SECONDS); - - assert fut != null; - assert fut.size() == 2; - - boolean ok = true; - - try { - fut.get(0).get(); - - ok = false; - } - catch (CancellationException e) { - info("First timeout task is cancelled: " + e.getMessage()); - } - - assert ok : "First task must be cancelled."; - - try { - fut.get(1).get(); - - ok = false; - } - catch (CancellationException e) { - info("Second timeout task is cancelled: " + e.getMessage()); - } - - assert ok : "Second task must be cancelled."; - - srvc.shutdown(); - } - - /** - * @throws Exception Thrown in case of test failure. - */ - @SuppressWarnings("TooBroadScope") - public void testInvokeAny() throws Exception { - Ignite ignite = G.ignite(getTestGridName()); - - ExecutorService srvc = createExecutorService(ignite); - - Collection<Callable<String>> cmds = new ArrayList<>(2); - - String val1 = "test-value-1"; - String val2 = "test-value-2"; - - cmds.add(new TestCallable<>(val1)); - cmds.add(new TestCallable<>(val2)); - - String res = srvc.invokeAny(cmds); - - info("Result: " + res); - - assert val1.equals(res) : "Failed to get valid result: " + res; - - srvc.shutdown(); - } - - /** - * @throws Exception Thrown in case of test failure. - */ - @SuppressWarnings("TooBroadScope") - public void testInvokeAnyWithTimeout() throws Exception { - Ignite ignite = G.ignite(getTestGridName()); - - ExecutorService srvc = createExecutorService(ignite); - - Collection<Callable<Integer>> timeoutCmds = new ArrayList<>(2); - - timeoutCmds.add(new TestCallable<>(5000)); - timeoutCmds.add(new TestCallable<>(5000)); - - boolean ok = true; - - try { - srvc.invokeAny(timeoutCmds, 1, TimeUnit.SECONDS); - - ok = false; - } - catch (TimeoutException e) { - info("Task timeout elapsed: " + e.getMessage()); - } - - assert ok : "Timeout must be thrown."; - - srvc.shutdown(); - } - - /** - * @param ignite Grid instance. - * @return Thrown in case of test failure. - */ - private ExecutorService createExecutorService(Ignite ignite) { - assert ignite != null; - - return new GridExecutorService((ClusterGroupAdapter) ignite, log()); - } - - /** - * @param <T> Type of the {@link Callable} argument. - */ - private static class TestCallable<T> implements Callable<T>, Serializable { - /** */ - private T data; - - /** */ - @IgniteInstanceResource - private Ignite ignite; - - /** - * @param data Data. - */ - TestCallable(T data) { - this.data = data; - } - - /** {@inheritDoc} */ - @Override public T call() throws Exception { - System.out.println("Test callable message."); - - assert ignite != null; - - if (data instanceof Integer) - Thread.sleep((Integer)data); - - return data; - } - } - - /** */ - private static class TestRunnable implements Runnable, Serializable { - /** */ - @IgniteInstanceResource - private Ignite ignite; - - /** {@inheritDoc} */ - @Override public void run() { - System.out.println("Test Runnable message."); - - assert ignite != null; - } - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/GridExplicitImplicitDeploymentSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridExplicitImplicitDeploymentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridExplicitImplicitDeploymentSelfTest.java deleted file mode 100644 index f19622e..0000000 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridExplicitImplicitDeploymentSelfTest.java +++ /dev/null @@ -1,476 +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; - -import org.apache.ignite.*; -import org.apache.ignite.cluster.*; -import org.apache.ignite.compute.*; -import org.apache.ignite.configuration.*; -import org.apache.ignite.resources.*; -import org.apache.ignite.internal.util.typedef.*; -import org.apache.ignite.internal.util.typedef.internal.*; -import org.apache.ignite.testframework.*; -import org.apache.ignite.testframework.junits.common.*; - -import java.io.*; -import java.util.*; - -/** - * - */ -@GridCommonTest(group = "Kernal Self") -public class GridExplicitImplicitDeploymentSelfTest extends GridCommonAbstractTest { - /** */ - public GridExplicitImplicitDeploymentSelfTest() { - super(/*start grid*/false); - } - - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - - // Override P2P configuration to exclude Task and Job classes - cfg.setPeerClassLoadingLocalClassPathExclude(GridDeploymentResourceTestTask.class.getName(), - GridDeploymentResourceTestJob.class.getName()); - - cfg.setDeploymentMode(IgniteDeploymentMode.ISOLATED); - - return cfg; - } - - /** - * @throws Exception If test failed. - */ - public void testImplicitDeployLocally() throws Exception { - execImplicitDeployLocally(true, true, true); - } - - /** - * @throws Exception If test failed. - */ - public void testImplicitDeployP2P() throws Exception { - execImplicitDeployP2P(true, true, true); - } - - /** - * @throws Exception If test failed. - */ - public void testExplicitDeployLocally() throws Exception { - execExplicitDeployLocally(true, true, true); - } - - /** - * @throws Exception If test failed. - */ - public void testExplicitDeployP2P() throws Exception { - execExplicitDeployP2P(true, true, true); - } - - /** - * @param ignite Grid. - */ - @SuppressWarnings({"CatchGenericClass"}) - private void stopGrid(Ignite ignite) { - try { - if (ignite != null) - G.stop(ignite.name(), true); - } - catch (Throwable e) { - error("Got error when stopping grid.", e); - } - } - - /** - * @param byCls If {@code true} than executes task by Class. - * @param byTask If {@code true} than executes task instance. - * @param byName If {@code true} than executes task by class name. - * @throws Exception If test failed. - */ - @SuppressWarnings("unchecked") - private void execExplicitDeployLocally(boolean byCls, boolean byTask, boolean byName) throws Exception { - Ignite ignite = null; - - try { - ignite = startGrid(); - - // Explicit Deployment. Task execution should return 0. - // Say resource class loader - different to task one. - ClassLoader ldr1 = new GridTestClassLoader( - Collections.singletonMap("testResource", "1"), - getClass().getClassLoader()); - - // Assume that users task and job were loaded with this class loader - ClassLoader ldr2 = new GridTestClassLoader( - Collections.singletonMap("testResource", "2"), - getClass().getClassLoader(), - GridDeploymentResourceTestTask.class.getName(), - GridDeploymentResourceTestJob.class.getName() - ); - - info("Loader1: " + ldr1); - info("Loader2: " + ldr2); - - Class<? extends ComputeTask<String, Integer>> taskCls = (Class<? extends ComputeTask<String, Integer>>) - ldr2.loadClass(GridDeploymentResourceTestTask.class.getName()); - - // Check auto-deploy. It should pick up resource class loader. - if (byCls) { - ignite.compute().localDeployTask(taskCls, ldr1); - - Integer res = ignite.compute().execute(taskCls, null); - - assert res != null; - assert res == 2 : "Invalid response: " + res; - } - - if (byTask) { - ignite.compute().localDeployTask(taskCls, ldr1); - - Integer res = ignite.compute().execute(taskCls.newInstance(), null); - - assert res != null; - assert res == 2 : "Invalid response: " + res; - } - - if (byName) { - ignite.compute().localDeployTask(taskCls, ldr1); - - Integer res = (Integer) ignite.compute().execute(taskCls.getName(), null); - - assert res != null; - assert res == 1 : "Invalid response: " + res; - } - } - finally { - stopGrid(ignite); - } - } - - /** - * @param byCls If {@code true} than executes task by Class. - * @param byTask If {@code true} than executes task instance. - * @param byName If {@code true} than executes task by class name. - * @throws Exception If test failed. - */ - @SuppressWarnings("unchecked") - private void execImplicitDeployLocally(boolean byCls, boolean byTask, boolean byName) throws Exception { - Ignite ignite = null; - - try { - ignite = startGrid(); - - // First task class loader. - ClassLoader ldr1 = new GridTestClassLoader( - Collections.singletonMap("testResource", "1"), - getClass().getClassLoader(), - GridDeploymentResourceTestTask.class.getName(), - GridDeploymentResourceTestJob.class.getName() - ); - - // Second task class loader - ClassLoader ldr2 = new GridTestClassLoader( - Collections.singletonMap("testResource", "2"), - getClass().getClassLoader(), - GridDeploymentResourceTestTask.class.getName(), - GridDeploymentResourceTestJob.class.getName() - ); - - // The same name but different classes/ class loaders. - Class<? extends ComputeTask<String, Integer>> taskCls1 = (Class<? extends ComputeTask<String, Integer>>) - ldr1.loadClass(GridDeploymentResourceTestTask.class.getName()); - - Class<? extends ComputeTask<String, Integer>> taskCls2 = (Class<? extends ComputeTask<String, Integer>>) - ldr2.loadClass(GridDeploymentResourceTestTask.class.getName()); - - if (byCls) { - Integer res1 = ignite.compute().execute(taskCls1, null); - Integer res2 = ignite.compute().execute(taskCls2, null); - - assert res1 != null; - assert res2 != null; - - assert res1 == 1 : "Invalid res1: " + res1; - assert res2 == 2 : "Invalid res2: " + res2; - } - - if (byTask) { - Integer res1 = ignite.compute().execute(taskCls1.newInstance(), null); - Integer res2 = ignite.compute().execute(taskCls2.newInstance(), null); - - assert res1 != null; - assert res2 != null; - - assert res1 == 1 : "Invalid res1: " + res1; - assert res2 == 2 : "Invalid res2: " + res2; - } - - if (byName) { - ignite.compute().localDeployTask(taskCls1, ldr1); - - Integer res1 = (Integer) ignite.compute().execute(taskCls1.getName(), null); - - ignite.compute().localDeployTask(taskCls2, ldr2); - - Integer res2 = (Integer) ignite.compute().execute(taskCls2.getName(), null); - - assert res1 != null; - assert res2 != null; - - assert res1 == 1 : "Invalid res1: " + res1; - assert res2 == 2 : "Invalid res2: " + res2; - } - } - finally { - stopGrid(ignite); - } - } - - /** - * @param byCls If {@code true} than executes task by Class. - * @param byTask If {@code true} than executes task instance. - * @param byName If {@code true} than executes task by class name. - * @throws Exception If test failed. - */ - @SuppressWarnings("unchecked") - private void execExplicitDeployP2P(boolean byCls, boolean byTask, boolean byName) throws Exception { - Ignite ignite1 = null; - Ignite ignite2 = null; - - try { - ignite1 = startGrid(1); - ignite2 = startGrid(2); - - ClassLoader ldr1 = new GridTestClassLoader( - Collections.singletonMap("testResource", "1"), - getClass().getClassLoader(), - GridDeploymentResourceTestTask.class.getName(), - GridDeploymentResourceTestJob.class.getName() - ); - - ClassLoader ldr2 = new GridTestClassLoader( - Collections.singletonMap("testResource", "2"), - getClass().getClassLoader(), - GridDeploymentResourceTestTask.class.getName(), - GridDeploymentResourceTestJob.class.getName() - ); - - Class<? extends ComputeTask<String, Integer>> taskCls = (Class<? extends ComputeTask<String, Integer>>) - ldr2.loadClass(GridDeploymentResourceTestTask.class.getName()); - - if (byCls) { - ignite1.compute().localDeployTask(taskCls, ldr1); - - // Even though the task is deployed with resource class loader, - // when we execute it, it will be redeployed with task class-loader. - Integer res = ignite1.compute().execute(taskCls, null); - - assert res != null; - assert res == 2 : "Invalid response: " + res; - } - - - if (byTask) { - ignite1.compute().localDeployTask(taskCls, ldr1); - - // Even though the task is deployed with resource class loader, - // when we execute it, it will be redeployed with task class-loader. - Integer res = ignite1.compute().execute(taskCls.newInstance(), null); - - assert res != null; - assert res == 2 : "Invalid response: " + res; - } - - if (byName) { - ignite1.compute().localDeployTask(taskCls, ldr1); - - // Even though the task is deployed with resource class loader, - // when we execute it, it will be redeployed with task class-loader. - Integer res = (Integer) ignite1.compute().execute(taskCls.getName(), null); - - assert res != null; - assert res == 1 : "Invalid response: " + res; - } - } - finally { - stopGrid(ignite2); - stopGrid(ignite1); - } - } - - /** - * @param byCls If {@code true} than executes task by Class. - * @param byTask If {@code true} than executes task instance. - * @param byName If {@code true} than executes task by class name. - * @throws Exception If test failed. - */ - @SuppressWarnings("unchecked") - private void execImplicitDeployP2P(boolean byCls, boolean byTask, boolean byName) throws Exception { - Ignite ignite1 = null; - Ignite ignite2 = null; - - try { - ignite1 = startGrid(1); - ignite2 = startGrid(2); - - ClassLoader ldr1 = new GridTestClassLoader( - Collections.singletonMap("testResource", "1"), - getClass().getClassLoader(), - GridDeploymentResourceTestTask.class.getName(), - GridDeploymentResourceTestJob.class.getName() - ); - - ClassLoader ldr2 = new GridTestClassLoader( - Collections.singletonMap("testResource", "2"), - getClass().getClassLoader(), - GridDeploymentResourceTestTask.class.getName(), - GridDeploymentResourceTestJob.class.getName() - ); - - Class<? extends ComputeTask<String, Integer>> taskCls1 = (Class<? extends ComputeTask<String, Integer>>) - ldr1.loadClass(GridDeploymentResourceTestTask.class.getName()); - - Class<? extends ComputeTask<String, Integer>> taskCls2 = (Class<? extends ComputeTask<String, Integer>>) - ldr2.loadClass(GridDeploymentResourceTestTask.class.getName()); - - if (byCls) { - Integer res1 = ignite1.compute().execute(taskCls1, null); - Integer res2 = ignite1.compute().execute(taskCls2, null); - - assert res1 != null; - assert res2 != null; - - assert res1 == 1 : "Invalid res1: " + res1; - assert res2 == 2 : "Invalid res2: " + res2; - } - - if (byTask) { - Integer res1 = ignite1.compute().execute(taskCls1.newInstance(), null); - Integer res2 = ignite1.compute().execute(taskCls2.newInstance(), null); - - assert res1 != null; - assert res2 != null; - - assert res1 == 1 : "Invalid res1: " + res1; - assert res2 == 2 : "Invalid res2: " + res2; - } - - if (byName) { - ignite1.compute().localDeployTask(taskCls1, ldr1); - - Integer res1 = (Integer) ignite1.compute().execute(taskCls1.getName(), null); - - ignite1.compute().localDeployTask(taskCls2, ldr2); - - Integer res2 = (Integer) ignite1.compute().execute(taskCls2.getName(), null); - - assert res1 != null; - assert res2 != null; - - assert res1 == 1 : "Invalid res1: " + res1; - assert res2 == 2 : "Invalid res2: " + res2; - } - } - finally { - stopGrid(ignite1); - stopGrid(ignite2); - } - } - - /** - * We use custom name to avoid auto-deployment in the same VM. - */ - @SuppressWarnings({"PublicInnerClass"}) - @ComputeTaskName("GridDeploymentResourceTestTask") - public static class GridDeploymentResourceTestTask extends ComputeTaskAdapter<String, Integer> { - /** */ - @IgniteInstanceResource - private Ignite ignite; - - /** {@inheritDoc} */ - @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, String arg) throws IgniteCheckedException { - Map<ComputeJobAdapter, ClusterNode> map = new HashMap<>(subgrid.size()); - - boolean ignoreLocNode = false; - - UUID locId = ignite.configuration().getNodeId(); - - if (subgrid.size() == 1) - assert subgrid.get(0).id().equals(locId) : "Wrong node id."; - else - ignoreLocNode = true; - - for (ClusterNode node : subgrid) { - // Ignore local node. - if (ignoreLocNode && node.id().equals(locId)) - continue; - - map.put(new GridDeploymentResourceTestJob(), node); - } - - return map; - } - - /** {@inheritDoc} */ - @Override public Integer reduce(List<ComputeJobResult> results) throws IgniteCheckedException { - return results.get(0).getData(); - } - } - - /** - * Simple job for this test. - */ - @SuppressWarnings({"PublicInnerClass"}) - public static final class GridDeploymentResourceTestJob extends ComputeJobAdapter { - /** */ - @IgniteLoggerResource - private IgniteLogger log; - - /** {@inheritDoc} */ - @Override public Serializable execute() throws IgniteCheckedException { - if (log.isInfoEnabled()) - log.info("Executing grid job: " + this); - - try { - ClassLoader ldr = Thread.currentThread().getContextClassLoader(); - - if (log.isInfoEnabled()) - log.info("Loader (inside job): " + ldr); - - InputStream in = ldr.getResourceAsStream("testResource"); - - if (in != null) { - Reader reader = new InputStreamReader(in); - - try { - char res = (char)reader.read(); - - return Integer.parseInt(Character.toString(res)); - } - finally { - U.close(in, null); - } - } - - return null; - } - catch (IOException e) { - throw new IgniteCheckedException("Failed to execute job.", e); - } - } - } -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/GridKernalTestUtils.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridKernalTestUtils.java b/modules/core/src/test/java/org/apache/ignite/internal/GridKernalTestUtils.java index 6a473f9..0c92b43 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridKernalTestUtils.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridKernalTestUtils.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal; import org.apache.ignite.Ignite; -import org.apache.ignite.internal.*; /** * Test kernal utils. @@ -40,6 +39,6 @@ public class GridKernalTestUtils { public static GridKernalContext context(Ignite ignite) { assert ignite != null; - return ((GridKernal) ignite).context(); + return ((IgniteKernal) ignite).context(); } } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/GridListenActorSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridListenActorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridListenActorSelfTest.java index 96926f5..343e48d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridListenActorSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridListenActorSelfTest.java @@ -19,7 +19,6 @@ package org.apache.ignite.internal; import org.apache.ignite.*; import org.apache.ignite.cluster.*; -import org.apache.ignite.internal.*; import org.apache.ignite.messaging.*; import org.apache.ignite.internal.util.typedef.*; import org.apache.ignite.testframework.*; @@ -52,7 +51,7 @@ public class GridListenActorSelfTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @SuppressWarnings("deprecation") @Override protected void afterTest() throws Exception { - ((GridKernal)grid()).context().io(). + ((IgniteKernal)grid()).context().io(). removeMessageListener(GridTopic.TOPIC_COMM_USER.name()); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/GridTopicExternalizableSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridTopicExternalizableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridTopicExternalizableSelfTest.java index b17bd1b..9d97559 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridTopicExternalizableSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridTopicExternalizableSelfTest.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal; import org.apache.ignite.*; -import org.apache.ignite.internal.*; import org.apache.ignite.lang.*; import org.apache.ignite.marshaller.*; @@ -28,7 +27,7 @@ import java.util.*; /** * Grid topic externalization test. */ -public class GridTopicExternalizableSelfTest extends GridExternalizableAbstractTest { +public class GridTopicExternalizableSelfTest extends IgniteExternalizableAbstractTest { /** */ private static final IgniteUuid A_GRID_UUID = IgniteUuid.randomUuid(); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/GridTopologyBuildVersionSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridTopologyBuildVersionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridTopologyBuildVersionSelfTest.java index 686640c..78b866a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridTopologyBuildVersionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridTopologyBuildVersionSelfTest.java @@ -80,7 +80,7 @@ public class GridTopologyBuildVersionSelfTest extends GridCommonAbstractTest { try { for (int i = 3; i >= 0; i--) { - GridKernal g = (GridKernal)grid(i); + IgniteKernal g = (IgniteKernal)grid(i); NavigableMap<IgniteProductVersion, Collection<ClusterNode>> verMap = g.context().discovery() .topologyVersionMap(); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java index 6edfa1b..51142d0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java @@ -17,7 +17,6 @@ package org.apache.ignite.internal; -import org.apache.ignite.internal.*; import org.apache.ignite.testframework.junits.common.*; import org.jetbrains.annotations.*; @@ -38,7 +37,7 @@ public class GridUpdateNotifierSelfTest extends GridCommonAbstractTest { */ public void testNotifier() throws Exception { GridUpdateNotifier ntf = new GridUpdateNotifier(null, GridProperties.get("ignite.version"), - GridKernal.SITE, TEST_GATEWAY, false); + IgniteKernal.SITE, TEST_GATEWAY, false); ntf.checkForNewVersion(new SelfExecutor(), log);